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/05/13 15:12:31 UTC

incubator-ignite git commit: # ignite-648: ignite configuration exchange

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-648 3cc9f7f46 -> cf26ffb78


# ignite-648: ignite configuration exchange


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

Branch: refs/heads/ignite-648
Commit: cf26ffb78c4980d0178bab58e73e1a5138ba99de
Parents: 3cc9f7f
Author: Artem Shutak <as...@gridgain.com>
Authored: Wed May 13 16:12:57 2015 +0300
Committer: Artem Shutak <as...@gridgain.com>
Committed: Wed May 13 16:12:57 2015 +0300

----------------------------------------------------------------------
 modules/core/pom.xml                            |   7 +
 .../cache/GridCacheAbstractFullApiSelfTest.java |   9 +-
 .../cache/multijvm/IgniteCacheProcessProxy.java | 500 -------------------
 .../cache/multijvm/IgniteExProcessProxy.java    | 450 -----------------
 .../cache/multijvm/IgniteNodeRunner.java        | 156 ------
 .../framework/IgniteCacheProcessProxy.java      | 500 +++++++++++++++++++
 .../framework/IgniteExProcessProxy.java         | 450 +++++++++++++++++
 .../multijvm/framework/IgniteNodeRunner.java    |  97 ++++
 8 files changed, 1061 insertions(+), 1108 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cf26ffb7/modules/core/pom.xml
----------------------------------------------------------------------
diff --git a/modules/core/pom.xml b/modules/core/pom.xml
index cdf2cf2..6257a4a 100644
--- a/modules/core/pom.xml
+++ b/modules/core/pom.xml
@@ -55,6 +55,13 @@
         </dependency>
 
         <dependency>
+            <groupId>com.thoughtworks.xstream</groupId>
+            <artifactId>xstream</artifactId>
+            <version>1.4.8</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
             <groupId>commons-cli</groupId>
             <artifactId>commons-cli</artifactId>
             <version>1.2</version>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cf26ffb7/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 801d752..67440e5 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,7 @@ 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.*;
+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,6 +35,8 @@ 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.multicast.*;
 import org.apache.ignite.spi.swapspace.inmemory.*;
 import org.apache.ignite.testframework.*;
 import org.apache.ignite.transactions.*;
@@ -62,6 +64,9 @@ 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 ipFinder = new TcpDiscoveryMulticastIpFinder();
+
     /** 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) {
@@ -147,7 +152,7 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
             cfg.setSwapSpaceSpi(new GridTestSwapSpaceSpi());
 
         if (isMultiJvm()) {
-            ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IgniteNodeRunner.ipFinder);
+            ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
 
             cfg.setLocalEventListeners(new HashMap<IgnitePredicate<? extends Event>, int[]>() {{
                 put(nodeJoinLsnr, new int[] {EventType.EVT_NODE_JOINED});

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cf26ffb7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/IgniteCacheProcessProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/IgniteCacheProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/IgniteCacheProcessProxy.java
deleted file mode 100644
index 0b203d9..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/IgniteCacheProcessProxy.java
+++ /dev/null
@@ -1,500 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.multijvm;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cache.*;
-import org.apache.ignite.cache.query.*;
-import org.apache.ignite.cluster.*;
-import org.apache.ignite.lang.*;
-import org.apache.ignite.mxbean.*;
-import org.jetbrains.annotations.*;
-
-import javax.cache.*;
-import javax.cache.CacheManager;
-import javax.cache.configuration.*;
-import javax.cache.expiry.*;
-import javax.cache.integration.*;
-import javax.cache.processor.*;
-import java.util.*;
-import java.util.concurrent.locks.*;
-
-/**
- * Ignite cache proxy for ignite instance at another JVM.
- */
-public class IgniteCacheProcessProxy<K, V> implements IgniteCache<K, V> {
-    /** Compute. */
-    private final transient IgniteCompute compute;
-
-    /** Cache name. */
-    private final String cacheName;
-
-    /** Grid id. */
-    private final UUID gridId;
-
-    /**
-     * @param name Name.
-     * @param proxy Ignite Process Proxy.
-     */
-    public IgniteCacheProcessProxy(String name, IgniteExProcessProxy proxy) {
-        cacheName = name;
-        gridId = proxy.getId();
-
-        ClusterGroup grp = proxy.localJvmGrid().cluster().forNodeId(proxy.getId());
-
-        compute = proxy.localJvmGrid().compute(grp);
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteCache<K, V> withAsync() {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean isAsync() {
-        return false; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public <R> IgniteFuture<R> future() {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public <C extends Configuration<K, V>> C getConfiguration(Class<C> clazz) {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public Entry<K, V> randomEntry() {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteCache<K, V> withExpiryPolicy(ExpiryPolicy plc) {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteCache<K, V> withSkipStore() {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public void loadCache(@Nullable IgniteBiPredicate<K, V> p, @Nullable Object... args) throws CacheException {
-        // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public void localLoadCache(@Nullable IgniteBiPredicate<K, V> p, @Nullable Object... args) throws CacheException {
-        // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public V getAndPutIfAbsent(final K key, final V val) throws CacheException {
-        return (V)compute.call(new IgniteCallable<Object>() {
-            @Override public Object call() throws Exception {
-                return Ignition.ignite(gridId).cache(cacheName).getAndPutIfAbsent(key, val);
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public Lock lock(K key) {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public Lock lockAll(Collection<? extends K> keys) {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean isLocalLocked(K key, boolean byCurrThread) {
-        return false; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public <R> QueryCursor<R> query(Query<R> qry) {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Override public Iterable<Entry<K, V>> localEntries(CachePeekMode... peekModes) throws CacheException {
-        // TODO: implement.
-//        return F.first(compute.broadcast(new IgniteClosureX<CachePeekMode[], Iterable>() {
-//            @Override public Iterable applyx(CachePeekMode... modes) {
-//                return Ignition.ignite(gridId).cache(cacheName).localEntries(modes);
-//            }
-//        }, peekModes));
-
-        return Collections.emptyList();
-    }
-
-    /** {@inheritDoc} */
-    @Override public QueryMetrics queryMetrics() {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public void localEvict(Collection<? extends K> keys) {
-        // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public V localPeek(final K key, final CachePeekMode... peekModes) {
-        return (V)compute.call(new IgniteCallable<Object>() {
-            @Override public Object call() throws Exception {
-                return Ignition.ignite(gridId).cache(cacheName).localPeek(key, peekModes);
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public void localPromote(Set<? extends K> keys) throws CacheException {
-        // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public int size(final CachePeekMode... peekModes) throws CacheException {
-        return (int)compute.call(new IgniteCallable<Object>() {
-            @Override public Object call() throws Exception {
-                return Ignition.ignite(gridId).cache(cacheName).size(peekModes);
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public int localSize(final CachePeekMode... peekModes) {
-        return (int)compute.call(new IgniteCallable<Object>() {
-            @Override public Object call() throws Exception {
-                return Ignition.ignite(gridId).cache(cacheName).localSize(peekModes);
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override  public <T> Map<K, EntryProcessorResult<T>> invokeAll(Map<? extends K, ? extends EntryProcessor<K, V, T>> map,
-        Object... args) {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public V get(final K key) {
-        return (V)compute.call(new IgniteCallable<Object>() {
-            @Override public Object call() throws Exception {
-                return Ignition.ignite(gridId).cache(cacheName).get(key);
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public Map<K, V> getAll(final Set<? extends K> keys) {
-        return (Map<K, V>)compute.call(new IgniteCallable<Object>() {
-            @Override public Object call() throws Exception {
-                return Ignition.ignite(gridId).cache(cacheName).getAll(keys);
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean containsKey(final K key) {
-        return (boolean)compute.call(new IgniteCallable<Object>() {
-            @Override public Object call() throws Exception {
-                return Ignition.ignite(gridId).cache(cacheName).containsKey(key);
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override  public void loadAll(Set<? extends K> keys, boolean replaceExistingValues, CompletionListener completionListener) {
-        // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean containsKeys(final Set<? extends K> keys) {
-        return (boolean)compute.call(new IgniteCallable<Object>() {
-            @Override public Object call() throws Exception {
-                return Ignition.ignite(gridId).cache(cacheName).containsKeys(keys);
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public void put(final K key, final V val) {;
-        compute.run(new IgniteRunnable() {
-            @Override public void run() {
-                Ignition.ignite(gridId).cache(cacheName).put(key, val);
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public V getAndPut(final K key, final V val) {
-        return (V)compute.call(new IgniteCallable<Object>() {
-            @Override public Object call() throws Exception {
-                return Ignition.ignite(gridId).cache(cacheName).getAndPut(key, val);
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public void putAll(final Map<? extends K, ? extends V> map) {
-        compute.run(new IgniteRunnable() {
-            @Override public void run() {
-                Ignition.ignite(gridId).cache(cacheName).putAll(map);
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean putIfAbsent(final K key, final V val) {
-        return (boolean)compute.call(new IgniteCallable<Object>() {
-            @Override public Object call() throws Exception {
-                return Ignition.ignite(gridId).cache(cacheName).putIfAbsent(key, val);
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean remove(final K key) {
-        return (boolean)compute.call(new IgniteCallable<Object>() {
-            @Override public Object call() throws Exception {
-                return Ignition.ignite(gridId).cache(cacheName).remove(key);
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean remove(final K key, final V oldVal) {
-        return (boolean)compute.call(new IgniteCallable<Object>() {
-            @Override public Object call() throws Exception {
-                return Ignition.ignite(gridId).cache(cacheName).remove(key, oldVal);
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public V getAndRemove(final K key) {
-        return (V)compute.call(new IgniteCallable<Object>() {
-            @Override public Object call() throws Exception {
-                return Ignition.ignite(gridId).cache(cacheName).getAndRemove(key);
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean replace(final K key, final V oldVal, final V newVal) {
-        return (boolean)compute.call(new IgniteCallable<Object>() {
-            @Override public Object call() throws Exception {
-                return Ignition.ignite(gridId).cache(cacheName).replace(key, oldVal, newVal);
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean replace(final K key, final V val) {
-        return (boolean)compute.call(new IgniteCallable<Object>() {
-            @Override public Object call() throws Exception {
-                return Ignition.ignite(gridId).cache(cacheName).replace(key, val);
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public V getAndReplace(final K key, final V val) {
-        return (V)compute.call(new IgniteCallable<Object>() {
-            @Override public Object call() throws Exception {
-                return Ignition.ignite(gridId).cache(cacheName).getAndReplace(key, val);
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public void removeAll(final Set<? extends K> keys) {
-        compute.run(new IgniteRunnable() {
-            @Override public void run() {
-                Ignition.ignite(gridId).cache(cacheName).removeAll(keys);
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public void removeAll() {
-        compute.run(new IgniteRunnable() {
-            @Override public void run() {
-                Ignition.ignite(gridId).cache(cacheName).removeAll();
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public void clear() {
-        compute.run(new IgniteRunnable() {
-            @Override public void run() {
-                Ignition.ignite(gridId).cache(cacheName).clear();
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public void clear(final K key) {
-        compute.run(new IgniteRunnable() {
-            @Override public void run() {
-                Ignition.ignite(gridId).cache(cacheName).clear(key);
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public void clearAll(final Set<? extends K> keys) {
-        compute.run(new IgniteRunnable() {
-            @Override public void run() {
-                Ignition.ignite(gridId).cache(cacheName).clearAll(keys);
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public void localClear(final K key) {
-        compute.run(new IgniteRunnable() {
-            @Override public void run() {
-                Ignition.ignite(gridId).cache(cacheName).localClear(key);
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public void localClearAll(final Set<? extends K> keys) {
-        compute.run(new IgniteRunnable() {
-            @Override public void run() {
-                Ignition.ignite(gridId).cache(cacheName).localClearAll(keys);
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public <T> T invoke(final K key, final EntryProcessor<K, V, T> entryProcessor, final Object... arguments) {
-        return (T)compute.call(new IgniteCallable<Object>() {
-            @Override public Object call() throws Exception {
-                return Ignition.ignite(gridId).cache(cacheName).invoke(key,
-                    (EntryProcessor<Object, Object, Object>)entryProcessor, arguments);
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public <T> T invoke(final K key, final CacheEntryProcessor<K, V, T> entryProcessor, final Object... arguments) {
-        return (T)compute.call(new IgniteCallable<Object>() {
-            @Override public Object call() throws Exception {
-                return Ignition.ignite(gridId).cache(cacheName).invoke(key,
-                    (CacheEntryProcessor<Object, Object, Object>)entryProcessor, arguments);
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override  public <T> Map<K, EntryProcessorResult<T>> invokeAll(final Set<? extends K> keys, final EntryProcessor<K, V, T> entryProcessor,
-        final Object... args) {
-        return (Map<K, EntryProcessorResult<T>>)compute.call(new IgniteCallable<Object>() {
-            @Override public Object call() throws Exception {
-                return Ignition.ignite(gridId).cache(cacheName).invokeAll(keys,
-                    (EntryProcessor<Object, Object, Object>)entryProcessor, args);
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public String getName() {
-        return (String)compute.call(new IgniteCallable<Object>() {
-            @Override public Object call() throws Exception {
-                return Ignition.ignite(gridId).cache(cacheName).getName();
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public CacheManager getCacheManager() {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public void close() {
-        compute.run(new IgniteRunnable() {
-            @Override public void run() {
-                Ignition.ignite(gridId).cache(cacheName).close();
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean isClosed() {
-        return (boolean)compute.call(new IgniteCallable<Object>() {
-            @Override public Object call() throws Exception {
-                return Ignition.ignite(gridId).cache(cacheName).isClosed();
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public <T> T unwrap(Class<T> clazz) {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override  public void registerCacheEntryListener(CacheEntryListenerConfiguration<K, V> cacheEntryListenerConfiguration) {
-        // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override  public void deregisterCacheEntryListener(CacheEntryListenerConfiguration<K, V> cacheEntryListenerConfiguration) {
-        // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public Iterator<Entry<K, V>> iterator() {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public <T> Map<K, EntryProcessorResult<T>> invokeAll(Set<? extends K> keys,
-        CacheEntryProcessor<K, V, T> entryProcessor, Object... args) {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteFuture<?> rebalance() {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public CacheMetrics metrics() {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public CacheMetrics metrics(ClusterGroup grp) {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public CacheMetricsMXBean mxBean() {
-        return null; // TODO: CODE: implement.
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cf26ffb7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/IgniteExProcessProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/IgniteExProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/IgniteExProcessProxy.java
deleted file mode 100644
index e6946e3..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/IgniteExProcessProxy.java
+++ /dev/null
@@ -1,450 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.multijvm;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cache.affinity.*;
-import org.apache.ignite.cluster.*;
-import org.apache.ignite.configuration.*;
-import org.apache.ignite.internal.*;
-import org.apache.ignite.internal.cluster.*;
-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.*;
-import org.jetbrains.annotations.*;
-
-import java.util.*;
-import java.util.concurrent.*;
-
-/**
- * Ignite proxy for ignite instance at another JVM.
- */
-public class IgniteExProcessProxy implements IgniteEx {
-    /** Grid proxies. */
-    private transient static final Map<String, IgniteExProcessProxy> gridProxies = new HashMap<>();
-
-    /** Jvm process with ignite instance. */
-    private transient final GridJavaProcess proc;
-
-    /** Configuration. */
-    private transient final IgniteConfiguration cfg;
-
-    /** Local jvm grid. */
-    private transient final Ignite locJvmGrid;
-
-    /** Logger. */
-    private transient final IgniteLogger log;
-
-    /** Grid id. */
-    private final UUID id = UUID.randomUUID();
-
-    /** Compute. */
-    private transient final IgniteCompute compute;
-
-    /**
-     * @param cfg Configuration.
-     * @param log Logger.
-     * @param locJvmGrid Local jvm grid.
-     */
-    public IgniteExProcessProxy(final IgniteConfiguration cfg, final IgniteLogger log, final Ignite locJvmGrid)
-        throws Exception {
-        this.cfg = cfg;
-        this.locJvmGrid = locJvmGrid;
-        this.log = log.getLogger("jvm-" + id.toString().substring(0, id.toString().indexOf('-')));
-
-        IgniteNodeRunner.storeToFile(cfg.getCacheConfiguration()[0]);
-
-        List<String> jvmArgs = U.jvmArgs();
-
-        List<String> filteredJvmArgs = new ArrayList<>();
-
-        for (String arg : jvmArgs) {
-            if(!arg.toLowerCase().startsWith("-agentlib"))
-                filteredJvmArgs.add(arg);
-        }
-
-        proc = GridJavaProcess.exec(
-            IgniteNodeRunner.class,
-            IgniteNodeRunner.asParams(id, cfg), // Params.
-            this.log,
-            // Optional closure to be called each time wrapped process prints line to system.out or system.err.
-            new IgniteInClosure<String>() {
-                @Override public void apply(String s) {
-                    IgniteExProcessProxy.this.log.info(s);
-                }
-            },
-            null,
-            filteredJvmArgs, // JVM Args.
-            System.getProperty("surefire.test.class.path")
-        );
-
-        // TODO: delete sleep.
-        U.sleep(3_000);
-
-        gridProxies.put(cfg.getGridName(), this);
-
-        compute = locJvmGrid.compute(locJvmGrid.cluster().forNodeId(id));
-    }
-
-    /**
-     * @param gridName Grid name.
-     * @return Instance by name or <code>null</code>.
-     */
-    public static IgniteExProcessProxy get(String gridName) {
-        return gridProxies.get(gridName);
-    }
-
-    /**
-     * Kill all running processes.
-     *
-     * @throws Exception if failed.
-     */
-    public static void killAll() throws Exception {
-        for (IgniteExProcessProxy ignite : gridProxies.values())
-            try {
-                ignite.getProcess().kill();
-            }
-            catch (Exception e) {
-                U.error(ignite.log, "Killing failed.", e);
-            }
-
-        gridProxies.clear();
-    }
-
-    /**
-     * @return Local JVM grid instance.
-     */
-    public Ignite localJvmGrid() {
-        return locJvmGrid;
-    }
-
-    /**
-     * @return Grid id.
-     */
-    public UUID getId() {
-        return id;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String name() {
-        return cfg.getGridName();
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteLogger log() {
-        return log;
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteConfiguration configuration() {
-        return cfg;
-    }
-
-    /** {@inheritDoc} */
-    @Override public <K extends GridCacheUtilityKey, V> IgniteInternalCache<K, V> utilityCache() {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public <K, V> IgniteInternalCache<K, V> cachex(@Nullable String name) {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public <K, V> IgniteInternalCache<K, V> cachex() {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public Collection<IgniteInternalCache<?, ?>> cachesx(
-        @Nullable IgnitePredicate<? super IgniteInternalCache<?, ?>>... p) {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean eventUserRecordable(int type) {
-        return false; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean allEventsUserRecordable(int[] types) {
-        return false; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean isJmxRemoteEnabled() {
-        return false; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean isRestartEnabled() {
-        return false; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public IgniteFileSystem igfsx(@Nullable String name) {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public Hadoop hadoop() {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteClusterEx cluster() {
-        return (IgniteClusterEx)locJvmGrid.cluster();
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public String latestVersion() {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public ClusterNode localNode() {
-        return F.first(compute.broadcast(new IgniteClosureX<Object, ClusterNode>() {
-            @Override public ClusterNode applyx(Object o) {
-                return ((IgniteEx)Ignition.ignite(id)).localNode();
-            }
-        }, null));
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridKernalContext context() {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteCompute compute() {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteCompute compute(ClusterGroup grp) {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteMessaging message() {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteMessaging message(ClusterGroup grp) {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteEvents events() {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteEvents events(ClusterGroup grp) {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteServices services() {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteServices services(ClusterGroup grp) {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public ExecutorService executorService() {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public ExecutorService executorService(ClusterGroup grp) {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteProductVersion version() {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteScheduler scheduler() {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public <K, V> IgniteCache<K, V> createCache(CacheConfiguration<K, V> cacheCfg) {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public <K, V> IgniteCache<K, V> createCache(String cacheName) {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public <K, V> IgniteCache<K, V> getOrCreateCache(CacheConfiguration<K, V> cacheCfg) {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public <K, V> IgniteCache<K, V> getOrCreateCache(String cacheName) {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public <K, V> void addCacheConfiguration(CacheConfiguration<K, V> cacheCfg) {
-        // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public <K, V> IgniteCache<K, V> createCache(CacheConfiguration<K, V> cacheCfg,
-        NearCacheConfiguration<K, V> nearCfg) {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public <K, V> IgniteCache<K, V> getOrCreateCache(CacheConfiguration<K, V> cacheCfg,
-        NearCacheConfiguration<K, V> nearCfg) {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override  public <K, V> IgniteCache<K, V> createNearCache(@Nullable String cacheName, NearCacheConfiguration<K, V> nearCfg) {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public <K, V> IgniteCache<K, V> getOrCreateNearCache(@Nullable String cacheName,
-        NearCacheConfiguration<K, V> nearCfg) {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public void destroyCache(String cacheName) {
-        // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public <K, V> IgniteCache<K, V> cache(@Nullable final String name) {
-        return new IgniteCacheProcessProxy(name, this);
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteTransactions transactions() {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public <K, V> IgniteDataStreamer<K, V> dataStreamer(@Nullable String cacheName) {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteFileSystem fileSystem(String name) {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public Collection<IgniteFileSystem> fileSystems() {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override  public IgniteAtomicSequence atomicSequence(String name, long initVal, boolean create) throws IgniteException {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteAtomicLong atomicLong(String name, long initVal, boolean create) throws IgniteException {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public <T> IgniteAtomicReference<T> atomicReference(String name, @Nullable T initVal,
-        boolean create) throws IgniteException {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@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.
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteCountDownLatch countDownLatch(String name, int cnt, boolean autoDel,
-        boolean create) throws IgniteException {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public <T> IgniteQueue<T> queue(String name, int cap,
-        @Nullable CollectionConfiguration cfg) throws IgniteException {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public <T> IgniteSet<T> set(String name, @Nullable CollectionConfiguration cfg) throws IgniteException {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public <T extends IgnitePlugin> T plugin(String name) throws PluginNotFoundException {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public void close() throws IgniteException {
-        try {
-            getProcess().kill();
-        }
-        catch (Exception e) {
-            e.printStackTrace();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public <K> Affinity<K> affinity(String cacheName) {
-        return null; // TODO: CODE: implement.
-    }
-
-    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;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cf26ffb7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/IgniteNodeRunner.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/IgniteNodeRunner.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/IgniteNodeRunner.java
deleted file mode 100644
index 6d06796..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/IgniteNodeRunner.java
+++ /dev/null
@@ -1,156 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.multijvm;
-
-import org.apache.ignite.*;
-import org.apache.ignite.configuration.*;
-import org.apache.ignite.internal.util.*;
-import org.apache.ignite.internal.util.typedef.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.marshaller.optimized.*;
-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.testframework.junits.*;
-
-import java.io.*;
-import java.util.*;
-
-/**
- * Run ignite node.
- */
-public class IgniteNodeRunner {
-    /** VM ip finder for TCP discovery. */
-    public static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryMulticastIpFinder();
-
-    /** */
-    private static final String CACHE_CONFIGURATION_TMP_FILE = System.getProperty("java.io.tmpdir") +
-        File.separator + "cacheConfiguration.tmp";
-
-    /**
-     * Starts {@link Ignite} instance accorging to given arguments.
-     *
-     * @param args Arguments.
-     * @throws Exception If failed.
-     */
-    public static void main(String[] args) throws Exception {
-        try {
-            X.println(GridJavaProcess.PID_MSG_PREFIX + U.jvmPid());
-
-            X.println("Starting Ignite Node... Args" + Arrays.toString(args));
-
-            IgniteConfiguration cfg = configuration(args);
-
-            Ignition.start(cfg);
-        }
-        catch (Throwable e) {
-            e.printStackTrace();
-
-            System.exit(1);
-        }
-    }
-
-    /**
-     * @param id Grid id.
-     * @param cfg Configuration.
-     * @return Given paramethers as command line string arguments.
-     */
-    public static String asParams(UUID id, IgniteConfiguration cfg) {
-        return id.toString() + ' ' + cfg.getGridName();
-    }
-
-    /**
-     * @param args Command line args.
-     * @return Ignite configuration.
-     * @throws Exception If failed.
-     */
-    private static IgniteConfiguration configuration(String[] args) throws Exception {
-        // Parse args.
-        assert args != null && args.length >= 1;
-
-        final UUID nodeId = UUID.fromString(args[0]);
-        final String gridName = args[1];
-
-        // Configuration.
-        IgniteConfiguration cfg = GridAbstractTest.getConfiguration0(gridName, new IgniteTestResources(),
-            GridCachePartitionedMultiJvmFullApiSelfTest.class, false);
-
-        cfg.setGridLogger(null);
-
-        TcpDiscoverySpi disco = new TcpDiscoverySpi();
-
-//        disco.setMaxMissedHeartbeats(Integer.MAX_VALUE);
-
-        disco.setIpFinder(ipFinder);
-
-//        if (isDebug())
-//            disco.setAckTimeout(Integer.MAX_VALUE);
-
-        cfg.setDiscoverySpi(disco);
-
-        cfg.setCacheConfiguration(cacheConfiguration());
-
-        cfg.setMarshaller(new OptimizedMarshaller(false));
-////        ----------------
-////        if (offHeapValues())
-////            cfg.setSwapSpaceSpi(new GridTestSwapSpaceSpi());
-////        ----------------
-//        cfg.getTransactionConfiguration().setTxSerializableEnabled(true);
-//
-////        ---------------
-//        Special.
-        cfg.setLocalHost("127.0.0.1");
-
-        cfg.setIncludeProperties();
-
-        cfg.setNodeId(nodeId);
-
-        return cfg;
-    }
-
-    /**
-     * Stors given cache configuration to the file.
-     *
-     * @param cc Cache configuration.
-     * @throws IOException If exception.
-     */
-    public static void storeToFile(CacheConfiguration cc) throws IOException {
-        File ccfgTmpFile = new File(CACHE_CONFIGURATION_TMP_FILE);
-
-        // TODO: add file created check (and delete the file after tests).
-        boolean created = ccfgTmpFile.createNewFile();
-
-        try(ObjectOutputStream out = new ObjectOutputStream(new FileOutputStream(ccfgTmpFile))) {
-            out.writeObject(cc);
-        }
-    }
-
-    /**
-     * Reads cache configuration from the file.
-     *
-     * @return Cache configuration.
-     * @throws Exception If exception.
-     */
-    private static CacheConfiguration cacheConfiguration() throws Exception {
-        File ccfgTmpFile = new File(CACHE_CONFIGURATION_TMP_FILE);
-
-        try(ObjectInputStream in = new ObjectInputStream(new FileInputStream(ccfgTmpFile))) {
-            return (CacheConfiguration)in.readObject();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cf26ffb7/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
new file mode 100644
index 0000000..6f4bf18
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteCacheProcessProxy.java
@@ -0,0 +1,500 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.multijvm.framework;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.cache.query.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.mxbean.*;
+import org.jetbrains.annotations.*;
+
+import javax.cache.*;
+import javax.cache.CacheManager;
+import javax.cache.configuration.*;
+import javax.cache.expiry.*;
+import javax.cache.integration.*;
+import javax.cache.processor.*;
+import java.util.*;
+import java.util.concurrent.locks.*;
+
+/**
+ * Ignite cache proxy for ignite instance at another JVM.
+ */
+public class IgniteCacheProcessProxy<K, V> implements IgniteCache<K, V> {
+    /** Compute. */
+    private final transient IgniteCompute compute;
+
+    /** Cache name. */
+    private final String cacheName;
+
+    /** Grid id. */
+    private final UUID gridId;
+
+    /**
+     * @param name Name.
+     * @param proxy Ignite Process Proxy.
+     */
+    public IgniteCacheProcessProxy(String name, IgniteExProcessProxy proxy) {
+        cacheName = name;
+        gridId = proxy.getId();
+
+        ClusterGroup grp = proxy.localJvmGrid().cluster().forNodeId(proxy.getId());
+
+        compute = proxy.localJvmGrid().compute(grp);
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteCache<K, V> withAsync() {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isAsync() {
+        return false; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public <R> IgniteFuture<R> future() {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public <C extends Configuration<K, V>> C getConfiguration(Class<C> clazz) {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public Entry<K, V> randomEntry() {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteCache<K, V> withExpiryPolicy(ExpiryPolicy plc) {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteCache<K, V> withSkipStore() {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void loadCache(@Nullable IgniteBiPredicate<K, V> p, @Nullable Object... args) throws CacheException {
+        // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void localLoadCache(@Nullable IgniteBiPredicate<K, V> p, @Nullable Object... args) throws CacheException {
+        // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public V getAndPutIfAbsent(final K key, final V val) throws CacheException {
+        return (V)compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                return Ignition.ignite(gridId).cache(cacheName).getAndPutIfAbsent(key, val);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public Lock lock(K key) {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public Lock lockAll(Collection<? extends K> keys) {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isLocalLocked(K key, boolean byCurrThread) {
+        return false; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public <R> QueryCursor<R> query(Query<R> qry) {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public Iterable<Entry<K, V>> localEntries(CachePeekMode... peekModes) throws CacheException {
+        // TODO: implement.
+//        return F.first(compute.broadcast(new IgniteClosureX<CachePeekMode[], Iterable>() {
+//            @Override public Iterable applyx(CachePeekMode... modes) {
+//                return Ignition.ignite(gridId).cache(cacheName).localEntries(modes);
+//            }
+//        }, peekModes));
+
+        return Collections.emptyList();
+    }
+
+    /** {@inheritDoc} */
+    @Override public QueryMetrics queryMetrics() {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void localEvict(Collection<? extends K> keys) {
+        // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public V localPeek(final K key, final CachePeekMode... peekModes) {
+        return (V)compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                return Ignition.ignite(gridId).cache(cacheName).localPeek(key, peekModes);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public void localPromote(Set<? extends K> keys) throws CacheException {
+        // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public int size(final CachePeekMode... peekModes) throws CacheException {
+        return (int)compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                return Ignition.ignite(gridId).cache(cacheName).size(peekModes);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public int localSize(final CachePeekMode... peekModes) {
+        return (int)compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                return Ignition.ignite(gridId).cache(cacheName).localSize(peekModes);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override  public <T> Map<K, EntryProcessorResult<T>> invokeAll(Map<? extends K, ? extends EntryProcessor<K, V, T>> map,
+        Object... args) {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public V get(final K key) {
+        return (V)compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                return Ignition.ignite(gridId).cache(cacheName).get(key);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public Map<K, V> getAll(final Set<? extends K> keys) {
+        return (Map<K, V>)compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                return Ignition.ignite(gridId).cache(cacheName).getAll(keys);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean containsKey(final K key) {
+        return (boolean)compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                return Ignition.ignite(gridId).cache(cacheName).containsKey(key);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override  public void loadAll(Set<? extends K> keys, boolean replaceExistingValues, CompletionListener completionListener) {
+        // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean containsKeys(final Set<? extends K> keys) {
+        return (boolean)compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                return Ignition.ignite(gridId).cache(cacheName).containsKeys(keys);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public void put(final K key, final V val) {;
+        compute.run(new IgniteRunnable() {
+            @Override public void run() {
+                Ignition.ignite(gridId).cache(cacheName).put(key, val);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public V getAndPut(final K key, final V val) {
+        return (V)compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                return Ignition.ignite(gridId).cache(cacheName).getAndPut(key, val);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public void putAll(final Map<? extends K, ? extends V> map) {
+        compute.run(new IgniteRunnable() {
+            @Override public void run() {
+                Ignition.ignite(gridId).cache(cacheName).putAll(map);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean putIfAbsent(final K key, final V val) {
+        return (boolean)compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                return Ignition.ignite(gridId).cache(cacheName).putIfAbsent(key, val);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean remove(final K key) {
+        return (boolean)compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                return Ignition.ignite(gridId).cache(cacheName).remove(key);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean remove(final K key, final V oldVal) {
+        return (boolean)compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                return Ignition.ignite(gridId).cache(cacheName).remove(key, oldVal);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public V getAndRemove(final K key) {
+        return (V)compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                return Ignition.ignite(gridId).cache(cacheName).getAndRemove(key);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean replace(final K key, final V oldVal, final V newVal) {
+        return (boolean)compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                return Ignition.ignite(gridId).cache(cacheName).replace(key, oldVal, newVal);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean replace(final K key, final V val) {
+        return (boolean)compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                return Ignition.ignite(gridId).cache(cacheName).replace(key, val);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public V getAndReplace(final K key, final V val) {
+        return (V)compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                return Ignition.ignite(gridId).cache(cacheName).getAndReplace(key, val);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public void removeAll(final Set<? extends K> keys) {
+        compute.run(new IgniteRunnable() {
+            @Override public void run() {
+                Ignition.ignite(gridId).cache(cacheName).removeAll(keys);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public void removeAll() {
+        compute.run(new IgniteRunnable() {
+            @Override public void run() {
+                Ignition.ignite(gridId).cache(cacheName).removeAll();
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public void clear() {
+        compute.run(new IgniteRunnable() {
+            @Override public void run() {
+                Ignition.ignite(gridId).cache(cacheName).clear();
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public void clear(final K key) {
+        compute.run(new IgniteRunnable() {
+            @Override public void run() {
+                Ignition.ignite(gridId).cache(cacheName).clear(key);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public void clearAll(final Set<? extends K> keys) {
+        compute.run(new IgniteRunnable() {
+            @Override public void run() {
+                Ignition.ignite(gridId).cache(cacheName).clearAll(keys);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public void localClear(final K key) {
+        compute.run(new IgniteRunnable() {
+            @Override public void run() {
+                Ignition.ignite(gridId).cache(cacheName).localClear(key);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public void localClearAll(final Set<? extends K> keys) {
+        compute.run(new IgniteRunnable() {
+            @Override public void run() {
+                Ignition.ignite(gridId).cache(cacheName).localClearAll(keys);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> T invoke(final K key, final EntryProcessor<K, V, T> entryProcessor, final Object... arguments) {
+        return (T)compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                return Ignition.ignite(gridId).cache(cacheName).invoke(key,
+                    (EntryProcessor<Object, Object, Object>)entryProcessor, arguments);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> T invoke(final K key, final CacheEntryProcessor<K, V, T> entryProcessor, final Object... arguments) {
+        return (T)compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                return Ignition.ignite(gridId).cache(cacheName).invoke(key,
+                    (CacheEntryProcessor<Object, Object, Object>)entryProcessor, arguments);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override  public <T> Map<K, EntryProcessorResult<T>> invokeAll(final Set<? extends K> keys, final EntryProcessor<K, V, T> entryProcessor,
+        final Object... args) {
+        return (Map<K, EntryProcessorResult<T>>)compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                return Ignition.ignite(gridId).cache(cacheName).invokeAll(keys,
+                    (EntryProcessor<Object, Object, Object>)entryProcessor, args);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getName() {
+        return (String)compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                return Ignition.ignite(gridId).cache(cacheName).getName();
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public CacheManager getCacheManager() {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void close() {
+        compute.run(new IgniteRunnable() {
+            @Override public void run() {
+                Ignition.ignite(gridId).cache(cacheName).close();
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isClosed() {
+        return (boolean)compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                return Ignition.ignite(gridId).cache(cacheName).isClosed();
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> T unwrap(Class<T> clazz) {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override  public void registerCacheEntryListener(CacheEntryListenerConfiguration<K, V> cacheEntryListenerConfiguration) {
+        // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override  public void deregisterCacheEntryListener(CacheEntryListenerConfiguration<K, V> cacheEntryListenerConfiguration) {
+        // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public Iterator<Entry<K, V>> iterator() {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> Map<K, EntryProcessorResult<T>> invokeAll(Set<? extends K> keys,
+        CacheEntryProcessor<K, V, T> entryProcessor, Object... args) {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteFuture<?> rebalance() {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public CacheMetrics metrics() {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public CacheMetrics metrics(ClusterGroup grp) {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public CacheMetricsMXBean mxBean() {
+        return null; // TODO: CODE: implement.
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cf26ffb7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteExProcessProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteExProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteExProcessProxy.java
new file mode 100644
index 0000000..034d32c
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteExProcessProxy.java
@@ -0,0 +1,450 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.multijvm.framework;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.affinity.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.cluster.*;
+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.*;
+import org.jetbrains.annotations.*;
+
+import java.util.*;
+import java.util.concurrent.*;
+
+/**
+ * Ignite proxy for ignite instance at another JVM.
+ */
+public class IgniteExProcessProxy implements IgniteEx {
+    /** Grid proxies. */
+    private transient static final Map<String, IgniteExProcessProxy> gridProxies = new HashMap<>();
+
+    /** Jvm process with ignite instance. */
+    private transient final GridJavaProcess proc;
+
+    /** Configuration. */
+    private transient final IgniteConfiguration cfg;
+
+    /** Local jvm grid. */
+    private transient final Ignite locJvmGrid;
+
+    /** Logger. */
+    private transient final IgniteLogger log;
+
+    /** Grid id. */
+    private final UUID id = UUID.randomUUID();
+
+    /** Compute. */
+    private transient final IgniteCompute compute;
+
+    /**
+     * @param cfg Configuration.
+     * @param log Logger.
+     * @param locJvmGrid Local jvm grid.
+     */
+    public IgniteExProcessProxy(final IgniteConfiguration cfg, final IgniteLogger log, final Ignite locJvmGrid)
+        throws Exception {
+        this.cfg = cfg;
+        this.locJvmGrid = locJvmGrid;
+        this.log = log.getLogger("jvm-" + id.toString().substring(0, id.toString().indexOf('-')));
+
+        IgniteNodeRunner.storeToFile(cfg.setNodeId(id));
+
+        List<String> jvmArgs = U.jvmArgs();
+
+        List<String> filteredJvmArgs = new ArrayList<>();
+
+        for (String arg : jvmArgs) {
+            if(!arg.toLowerCase().startsWith("-agentlib"))
+                filteredJvmArgs.add(arg);
+        }
+
+        proc = GridJavaProcess.exec(
+            IgniteNodeRunner.class,
+            IgniteNodeRunner.asParams(id, cfg), // Params.
+            this.log,
+            // Optional closure to be called each time wrapped process prints line to system.out or system.err.
+            new IgniteInClosure<String>() {
+                @Override public void apply(String s) {
+                    IgniteExProcessProxy.this.log.info(s);
+                }
+            },
+            null,
+            filteredJvmArgs, // JVM Args.
+            System.getProperty("surefire.test.class.path")
+        );
+
+        // TODO: delete sleep.
+        U.sleep(3_000);
+
+        gridProxies.put(cfg.getGridName(), this);
+
+        compute = locJvmGrid.compute(locJvmGrid.cluster().forNodeId(id));
+    }
+
+    /**
+     * @param gridName Grid name.
+     * @return Instance by name or <code>null</code>.
+     */
+    public static IgniteExProcessProxy get(String gridName) {
+        return gridProxies.get(gridName);
+    }
+
+    /**
+     * Kill all running processes.
+     *
+     * @throws Exception if failed.
+     */
+    public static void killAll() throws Exception {
+        for (IgniteExProcessProxy ignite : gridProxies.values())
+            try {
+                ignite.getProcess().kill();
+            }
+            catch (Exception e) {
+                U.error(ignite.log, "Killing failed.", e);
+            }
+
+        gridProxies.clear();
+    }
+
+    /**
+     * @return Local JVM grid instance.
+     */
+    public Ignite localJvmGrid() {
+        return locJvmGrid;
+    }
+
+    /**
+     * @return Grid id.
+     */
+    public UUID getId() {
+        return id;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String name() {
+        return cfg.getGridName();
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteLogger log() {
+        return log;
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteConfiguration configuration() {
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override public <K extends GridCacheUtilityKey, V> IgniteInternalCache<K, V> utilityCache() {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public <K, V> IgniteInternalCache<K, V> cachex(@Nullable String name) {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public <K, V> IgniteInternalCache<K, V> cachex() {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<IgniteInternalCache<?, ?>> cachesx(
+        @Nullable IgnitePredicate<? super IgniteInternalCache<?, ?>>... p) {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean eventUserRecordable(int type) {
+        return false; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean allEventsUserRecordable(int[] types) {
+        return false; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isJmxRemoteEnabled() {
+        return false; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isRestartEnabled() {
+        return false; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public IgniteFileSystem igfsx(@Nullable String name) {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public Hadoop hadoop() {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteClusterEx cluster() {
+        return (IgniteClusterEx)locJvmGrid.cluster();
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public String latestVersion() {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterNode localNode() {
+        return F.first(compute.broadcast(new IgniteClosureX<Object, ClusterNode>() {
+            @Override public ClusterNode applyx(Object o) {
+                return ((IgniteEx)Ignition.ignite(id)).localNode();
+            }
+        }, null));
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridKernalContext context() {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteCompute compute() {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteCompute compute(ClusterGroup grp) {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteMessaging message() {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteMessaging message(ClusterGroup grp) {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteEvents events() {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteEvents events(ClusterGroup grp) {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteServices services() {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteServices services(ClusterGroup grp) {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public ExecutorService executorService() {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public ExecutorService executorService(ClusterGroup grp) {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteProductVersion version() {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteScheduler scheduler() {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public <K, V> IgniteCache<K, V> createCache(CacheConfiguration<K, V> cacheCfg) {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public <K, V> IgniteCache<K, V> createCache(String cacheName) {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public <K, V> IgniteCache<K, V> getOrCreateCache(CacheConfiguration<K, V> cacheCfg) {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public <K, V> IgniteCache<K, V> getOrCreateCache(String cacheName) {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public <K, V> void addCacheConfiguration(CacheConfiguration<K, V> cacheCfg) {
+        // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public <K, V> IgniteCache<K, V> createCache(CacheConfiguration<K, V> cacheCfg,
+        NearCacheConfiguration<K, V> nearCfg) {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public <K, V> IgniteCache<K, V> getOrCreateCache(CacheConfiguration<K, V> cacheCfg,
+        NearCacheConfiguration<K, V> nearCfg) {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override  public <K, V> IgniteCache<K, V> createNearCache(@Nullable String cacheName, NearCacheConfiguration<K, V> nearCfg) {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public <K, V> IgniteCache<K, V> getOrCreateNearCache(@Nullable String cacheName,
+        NearCacheConfiguration<K, V> nearCfg) {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void destroyCache(String cacheName) {
+        // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public <K, V> IgniteCache<K, V> cache(@Nullable final String name) {
+        return new IgniteCacheProcessProxy(name, this);
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteTransactions transactions() {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public <K, V> IgniteDataStreamer<K, V> dataStreamer(@Nullable String cacheName) {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteFileSystem fileSystem(String name) {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<IgniteFileSystem> fileSystems() {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override  public IgniteAtomicSequence atomicSequence(String name, long initVal, boolean create) throws IgniteException {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteAtomicLong atomicLong(String name, long initVal, boolean create) throws IgniteException {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> IgniteAtomicReference<T> atomicReference(String name, @Nullable T initVal,
+        boolean create) throws IgniteException {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@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.
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteCountDownLatch countDownLatch(String name, int cnt, boolean autoDel,
+        boolean create) throws IgniteException {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> IgniteQueue<T> queue(String name, int cap,
+        @Nullable CollectionConfiguration cfg) throws IgniteException {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> IgniteSet<T> set(String name, @Nullable CollectionConfiguration cfg) throws IgniteException {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T extends IgnitePlugin> T plugin(String name) throws PluginNotFoundException {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void close() throws IgniteException {
+        try {
+            getProcess().kill();
+        }
+        catch (Exception e) {
+            e.printStackTrace();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public <K> Affinity<K> affinity(String cacheName) {
+        return null; // TODO: CODE: implement.
+    }
+
+    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;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cf26ffb7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteNodeRunner.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteNodeRunner.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteNodeRunner.java
new file mode 100644
index 0000000..d927fc0
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteNodeRunner.java
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.multijvm.framework;
+
+import com.thoughtworks.xstream.*;
+import org.apache.ignite.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.util.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.marshaller.optimized.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.*;
+
+import java.io.*;
+import java.util.*;
+
+/**
+ * Run ignite node.
+ */
+public class IgniteNodeRunner {
+    /** */
+    private static final String IGNITE_CONFIGURATION_FILE = System.getProperty("java.io.tmpdir") +
+        File.separator + "igniteConfiguration.tmp";
+
+    /**
+     * Starts {@link Ignite} instance accorging to given arguments.
+     *
+     * @param args Arguments.
+     * @throws Exception If failed.
+     */
+    public static void main(String[] args) throws Exception {
+        try {
+            X.println(GridJavaProcess.PID_MSG_PREFIX + U.jvmPid());
+
+            X.println("Starting Ignite Node... Args" + Arrays.toString(args));
+
+            IgniteConfiguration cfg = readFromFile();
+
+            Ignition.start(cfg);
+        }
+        catch (Throwable e) {
+            e.printStackTrace();
+
+            System.exit(1);
+        }
+    }
+
+    /**
+     * @param id Grid id.
+     * @param cfg Configuration.
+     * @return Given paramethers as command line string arguments.
+     */
+    public static String asParams(UUID id, IgniteConfiguration cfg) {
+        return id.toString() + ' ' + cfg.getGridName();
+    }
+
+    public static void storeToFile(IgniteConfiguration cfg) throws IOException {
+        try(OutputStream out = new BufferedOutputStream(new FileOutputStream(IGNITE_CONFIGURATION_FILE))) {
+            cfg.setMBeanServer(null);
+            cfg.setMarshaller(null);
+            cfg.setDiscoverySpi(null);
+
+            new XStream().toXML(cfg, out);
+        }
+    }
+
+    private static IgniteConfiguration readFromFile() throws FileNotFoundException {
+        BufferedReader cfgReader = new BufferedReader(new FileReader(IGNITE_CONFIGURATION_FILE));
+        
+        IgniteConfiguration cfg = (IgniteConfiguration)new XStream().fromXML(cfgReader);
+        
+        cfg.setMarshaller(new OptimizedMarshaller(false));
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+        disco.setIpFinder(new TcpDiscoveryMulticastIpFinder());
+        
+        cfg.setDiscoverySpi(disco);
+        
+        return cfg;
+    }
+}