You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2015/06/23 21:20:17 UTC

[1/2] incubator-ignite git commit: # ignite-648: move framework

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


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b77db74b/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
new file mode 100644
index 0000000..6f9a0cb
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteEventsProcessProxy.java
@@ -0,0 +1,147 @@
+/*
+ * 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.testframework.junits.multijvm;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.events.*;
+import org.apache.ignite.lang.*;
+import org.jetbrains.annotations.*;
+
+import java.util.*;
+
+/**
+ * Ignite events proxy for ignite instance at another JVM.
+ */
+public class IgniteEventsProcessProxy implements IgniteEvents {
+    /** Ignite proxy. */
+    private final transient IgniteProcessProxy igniteProxy;
+
+    /** Grid id. */
+    private final UUID gridId;
+
+    /**
+     * @param igniteProxy Ignite proxy.
+     */
+    public IgniteEventsProcessProxy(IgniteProcessProxy igniteProxy) {
+        this.igniteProxy = igniteProxy;
+
+        gridId = igniteProxy.getId();
+    }
+
+    /**
+     * @return Events instance.
+     */
+    private IgniteEvents events() {
+        return Ignition.ignite(gridId).events();
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterGroup clusterGroup() {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T extends Event> List<T> remoteQuery(IgnitePredicate<T> p, long timeout,
+        @Nullable int... types) throws IgniteException {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@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.
+    }
+
+    /** {@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.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stopRemoteListen(UUID opId) throws IgniteException {
+        // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T extends Event> T waitForLocal(@Nullable IgnitePredicate<T> filter,
+        @Nullable int... types) throws IgniteException {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T extends Event> Collection<T> localQuery(IgnitePredicate<T> p, @Nullable int... types) {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void recordLocal(Event evt) {
+        // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void localListen(final IgnitePredicate<? extends Event> lsnr, final int... types) {
+        igniteProxy.remoteCompute().run(new IgniteRunnable() {
+            @Override public void run() {
+                events().localListen(lsnr, types);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean stopLocalListen(IgnitePredicate<? extends Event> lsnr, @Nullable int... types) {
+        return false; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void enableLocal(int... types) {
+        // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void disableLocal(int... types) {
+        // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public int[] enabledEvents() {
+        return new int[0]; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isEnabled(int type) {
+        return false; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteEvents 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.
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b77db74b/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
new file mode 100644
index 0000000..677d383
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteNodeRunner.java
@@ -0,0 +1,111 @@
+/*
+ * 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.testframework.junits.multijvm;
+
+import com.thoughtworks.xstream.*;
+import org.apache.ignite.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.processors.cache.*;
+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 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 = readCfgFromFileAndDeleteFile(args[0]);
+
+            Ignition.start(cfg);
+        }
+        catch (Throwable e) {
+            e.printStackTrace();
+
+            System.exit(1);
+        }
+    }
+
+    /**
+     * Stores {@link IgniteConfiguration} to file as xml.
+     *
+     * @param cfg Ignite Configuration.
+     * @return A name of file where the configuration was stored.
+     * @throws IOException If failed.
+     * @see #readCfgFromFileAndDeleteFile(String)
+     */
+    public static String storeToFile(IgniteConfiguration cfg) throws IOException {
+        String fileName = IGNITE_CONFIGURATION_FILE + cfg.getNodeId();
+
+        try(OutputStream out = new BufferedOutputStream(new FileOutputStream(fileName))) {
+            cfg.setMBeanServer(null);
+            cfg.setMarshaller(null);
+            cfg.setDiscoverySpi(null);
+            cfg.setGridLogger(null);
+
+            new XStream().toXML(cfg, out);
+        }
+
+        return fileName;
+    }
+
+    /**
+     * Reads configuration from given file and delete the file after.
+     *
+     * @param fileName File name.
+     * @return Readed configuration.
+     * @throws IOException If failed.
+     * @see #storeToFile(IgniteConfiguration)
+     */
+    private static IgniteConfiguration readCfgFromFileAndDeleteFile(String fileName) throws IOException {
+        try(BufferedReader cfgReader = new BufferedReader(new FileReader(fileName))) {
+            IgniteConfiguration cfg = (IgniteConfiguration)new XStream().fromXML(cfgReader);
+
+            cfg.setMarshaller(new OptimizedMarshaller(false));
+
+            TcpDiscoverySpi disco = new TcpDiscoverySpi();
+            disco.setIpFinder(GridCacheAbstractFullApiSelfTest.LOCAL_IP_FINDER);
+            cfg.setDiscoverySpi(disco);
+
+            return cfg;
+        }
+        finally {
+            new File(fileName).delete();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b77db74b/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
new file mode 100644
index 0000000..31c65f0
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
@@ -0,0 +1,467 @@
+/*
+ * 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.testframework.junits.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.events.*;
+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.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 IgniteProcessProxy implements IgniteEx {
+    /** Grid proxies. */
+    private transient static final Map<String, IgniteProcessProxy> 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();
+
+    /** Remote ignite instance started latch. */
+    private transient final CountDownLatch rmtNodeStartedLatch = new CountDownLatch(1);
+
+    /**
+     * @param cfg Configuration.
+     * @param log Logger.
+     * @param locJvmGrid Local jvm grid.
+     */
+    public IgniteProcessProxy(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('-')));
+
+        String cfgFileName = 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);
+        }
+
+        locJvmGrid.events().localListen(new IgnitePredicateX<Event>() {
+            @Override public boolean applyx(Event e) {
+                if (((DiscoveryEvent)e).eventNode().id().equals(id)) {
+                    rmtNodeStartedLatch.countDown();
+
+                    return false;
+                }
+
+                return true;
+            }
+        }, EventType.EVT_NODE_JOINED);
+
+        proc = GridJavaProcess.exec(
+            IgniteNodeRunner.class,
+            cfgFileName, // 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) {
+                    IgniteProcessProxy.this.log.info(s);
+                }
+            },
+            null,
+            filteredJvmArgs, // JVM Args.
+            System.getProperty("surefire.test.class.path")
+        );
+
+        assert rmtNodeStartedLatch.await(30, TimeUnit.SECONDS): "Remote node with id=" + id + " didn't join.";
+
+        gridProxies.put(cfg.getGridName(), this);
+    }
+
+    /**
+     * @param gridName Grid name.
+     * @return Instance by name or exception wiil be thrown.
+     */
+    public static IgniteProcessProxy get(String gridName) {
+        IgniteProcessProxy res = gridProxies.get(gridName);
+
+        if (res == null)
+            throw new IgniteIllegalStateException("Grid instance was not properly started " +
+                "or was already stopped: " + gridName);
+
+        return res;
+    }
+
+    /**
+     * Kill all running processes.
+     */
+    public static void killAll() {
+        for (IgniteProcessProxy 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 remoteCompute().call(new IgniteCallable<ClusterNode>() {
+            @Override public ClusterNode call() throws Exception {
+                return ((IgniteEx)Ignition.ignite(id)).localNode();
+            }
+        });
+    }
+
+    /** {@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 new IgniteEventsProcessProxy(this);
+    }
+
+    /** {@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() {
+        throw new UnsupportedOperationException("Transactions are not supported in multi JVM mode.");
+    }
+
+    /** {@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 new AffinityProcessProxy(cacheName, this);
+    }
+
+    /**
+     * @return Jvm process in which grid node started.
+     */
+    public GridJavaProcess getProcess() {
+        return proc;
+    }
+
+    /**
+     * @return {@link IgniteCompute} instance to communicate with remote node.
+     */
+    public IgniteCompute remoteCompute() {
+        ClusterGroup grp = locJvmGrid.cluster().forNodeId(id);
+
+        if (grp.nodes().isEmpty())
+            throw new IllegalStateException("Could not found node with id=" + id + ".");
+
+        return locJvmGrid.compute(grp);
+    }
+}


[2/2] incubator-ignite git commit: # ignite-648: move framework

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


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

Branch: refs/heads/ignite-648
Commit: b77db74b97797ed58d6deb0c823774aac2a2b551
Parents: ada57fc
Author: ashutak <as...@gridgain.com>
Authored: Tue Jun 23 22:20:38 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Tue Jun 23 22:20:38 2015 +0300

----------------------------------------------------------------------
 .../cache/GridCacheAbstractSelfTest.java        |   2 +-
 ...dCacheReplicatedMultiJvmFullApiSelfTest.java |   7 +
 .../framework/AffinityProcessProxy.java         | 194 ------
 .../framework/IgniteCacheProcessProxy.java      | 586 -------------------
 .../framework/IgniteEventsProcessProxy.java     | 147 -----
 .../multijvm/framework/IgniteNodeRunner.java    | 111 ----
 .../multijvm/framework/IgniteProcessProxy.java  | 467 ---------------
 .../testframework/junits/GridAbstractTest.java  |   2 +-
 .../junits/multijvm/AffinityProcessProxy.java   | 194 ++++++
 .../multijvm/IgniteCacheProcessProxy.java       | 586 +++++++++++++++++++
 .../multijvm/IgniteEventsProcessProxy.java      | 147 +++++
 .../junits/multijvm/IgniteNodeRunner.java       | 111 ++++
 .../junits/multijvm/IgniteProcessProxy.java     | 467 +++++++++++++++
 13 files changed, 1514 insertions(+), 1507 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b77db74b/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 d4c0305..37fc352 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
@@ -23,7 +23,6 @@ import org.apache.ignite.cache.store.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.cluster.*;
-import org.apache.ignite.internal.processors.cache.multijvm.framework.*;
 import org.apache.ignite.internal.util.lang.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
@@ -33,6 +32,7 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
 import org.apache.ignite.testframework.*;
 import org.apache.ignite.testframework.junits.common.*;
+import org.apache.ignite.testframework.junits.multijvm.*;
 import org.apache.ignite.transactions.*;
 import org.jetbrains.annotations.*;
 import org.jsr166.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b77db74b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheReplicatedMultiJvmFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheReplicatedMultiJvmFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheReplicatedMultiJvmFullApiSelfTest.java
index b135099..2280838 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheReplicatedMultiJvmFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheReplicatedMultiJvmFullApiSelfTest.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.cache.multijvm;
 
 import org.apache.ignite.internal.processors.cache.distributed.replicated.*;
+import org.apache.ignite.testframework.junits.multijvm.*;
 
 /**
  * Multy Jvm tests.
@@ -27,4 +28,10 @@ public class GridCacheReplicatedMultiJvmFullApiSelfTest extends GridCacheReplica
     protected boolean isMultiJvm() {
         return true;
     }
+
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        IgniteProcessProxy.killAll();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b77db74b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/AffinityProcessProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/AffinityProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/AffinityProcessProxy.java
deleted file mode 100644
index 7168534..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/AffinityProcessProxy.java
+++ /dev/null
@@ -1,194 +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.framework;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cache.affinity.*;
-import org.apache.ignite.cluster.*;
-import org.apache.ignite.lang.*;
-import org.jetbrains.annotations.*;
-
-import java.util.*;
-
-/**
- * Proxy class for affinity at another jvm.
- */
-public class AffinityProcessProxy<K> implements Affinity<K> {
-    /** Compute. */
-    private final transient IgniteCompute compute;
-
-    /** Cache name. */
-    private final String cacheName;
-
-    /** Grid id. */
-    private final UUID gridId;
-
-    /**
-     * @param cacheName Cache name.
-     * @param proxy Ignite ptocess proxy.
-     */
-    public AffinityProcessProxy(String cacheName, IgniteProcessProxy proxy) {
-        this.cacheName = cacheName;
-        gridId = proxy.getId();
-        compute = proxy.remoteCompute();
-    }
-
-    /**
-     * Returns cache instance. Method to be called from closure at another JVM.
-     *
-     * @return Cache.
-     */
-    private Affinity<Object> affinity() {
-        return Ignition.ignite(gridId).affinity(cacheName);
-    }
-
-    /** {@inheritDoc} */
-    @Override public int partitions() {
-        return (int)compute.call(new IgniteCallable<Object>() {
-            @Override public Object call() throws Exception {
-                return affinity().partitions();
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public int partition(final K key) {
-        return (int)compute.call(new IgniteCallable<Object>() {
-            @Override public Object call() throws Exception {
-                return affinity().partition(key);
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean isPrimary(final ClusterNode n, final K key) {
-        return (boolean)compute.call(new IgniteCallable<Object>() {
-            @Override public Object call() throws Exception {
-                return affinity().isPrimary(n, key);
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean isBackup(final ClusterNode n, final K key) {
-        return (boolean)compute.call(new IgniteCallable<Object>() {
-            @Override public Object call() throws Exception {
-                return affinity().isBackup(n, key);
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean isPrimaryOrBackup(final ClusterNode n, final K key) {
-        return (boolean)compute.call(new IgniteCallable<Object>() {
-            @Override public Object call() throws Exception {
-                return affinity().isPrimaryOrBackup(n, key);
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public int[] primaryPartitions(final ClusterNode n) {
-        return (int[])compute.call(new IgniteCallable<Object>() {
-            @Override public Object call() throws Exception {
-                return affinity().primaryPartitions(n);
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public int[] backupPartitions(final ClusterNode n) {
-        return (int[])compute.call(new IgniteCallable<Object>() {
-            @Override public Object call() throws Exception {
-                return affinity().backupPartitions(n);
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public int[] allPartitions(final ClusterNode n) {
-        return (int[])compute.call(new IgniteCallable<Object>() {
-            @Override public Object call() throws Exception {
-                return affinity().allPartitions(n);
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public Object affinityKey(final K key) {
-        return compute.call(new IgniteCallable<Object>() {
-            @Override public Object call() throws Exception {
-                return affinity().affinityKey(key);
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public Map<ClusterNode, Collection<K>> mapKeysToNodes(final Collection<? extends K> keys) {
-        return (Map<ClusterNode, Collection<K>>)compute.call(new IgniteCallable<Object>() {
-            @Override public Object call() throws Exception {
-                return affinity().mapKeysToNodes(keys);
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public ClusterNode mapKeyToNode(final K key) {
-        return (ClusterNode)compute.call(new IgniteCallable<Object>() {
-            @Override public Object call() throws Exception {
-                return affinity().mapKeyToNode(key);
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public Collection<ClusterNode> mapKeyToPrimaryAndBackups(final K key) {
-        return (Collection<ClusterNode>)compute.call(new IgniteCallable<Object>() {
-            @Override public Object call() throws Exception {
-                return affinity().mapKeyToPrimaryAndBackups(key);
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public ClusterNode mapPartitionToNode(final int part) {
-        return (ClusterNode)compute.call(new IgniteCallable<Object>() {
-            @Override public Object call() throws Exception {
-                return affinity().mapPartitionToNode(part);
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public Map<Integer, ClusterNode> mapPartitionsToNodes(final Collection<Integer> parts) {
-        return (Map<Integer, ClusterNode>)compute.call(new IgniteCallable<Object>() {
-            @Override public Object call() throws Exception {
-                return affinity().mapPartitionsToNodes(parts);
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public Collection<ClusterNode> mapPartitionToPrimaryAndBackups(final int part) {
-        return (Collection<ClusterNode>)compute.call(new IgniteCallable<Object>() {
-            @Override public Object call() throws Exception {
-                return affinity().mapPartitionToPrimaryAndBackups(part);
-            }
-        });
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b77db74b/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
deleted file mode 100644
index 27b2262..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteCacheProcessProxy.java
+++ /dev/null
@@ -1,586 +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.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;
-
-    /** With async. */
-    private final boolean isAsync;
-
-    /** Ignite proxy. */
-    private final transient IgniteProcessProxy igniteProxy;
-
-    /**
-     * @param name Name.
-     * @param proxy Ignite Process Proxy.
-     */
-    public IgniteCacheProcessProxy(String name, IgniteProcessProxy proxy) {
-        this(name, false, proxy);
-    }
-
-    /**
-     * @param name Name.
-     * @param async
-     * @param proxy Ignite Process Proxy.
-     */
-    public IgniteCacheProcessProxy(String name, boolean async, IgniteProcessProxy proxy) {
-        cacheName = name;
-        isAsync = async;
-        gridId = proxy.getId();
-        igniteProxy = proxy;
-        compute = proxy.remoteCompute();
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteCache<K, V> withAsync() {
-        return new IgniteCacheProcessProxy<>(cacheName, true, igniteProxy);
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean isAsync() {
-        return isAsync;
-    }
-
-    /** {@inheritDoc} */
-    @Override public <R> IgniteFuture<R> future() {
-        // TODO implement.
-//        R futureRes = (R)compute.call(new IgniteCallable<Object>() {
-//            @Override public Object call() throws Exception {
-//                return cache().future().get();
-//            }
-//        });
-//
-//        return new IgniteFinishedFutureImpl<R>(futureRes);
-
-        throw new UnsupportedOperationException("Method should be supported.");
-    }
-
-    /** {@inheritDoc} */
-    @Override public <C extends Configuration<K, V>> C getConfiguration(final Class<C> clazz) {
-        final Class cl = clazz;
-
-        return (C)compute.call(new IgniteCallable<Object>() {
-            @Override public Object call() throws Exception {
-                return cache().getConfiguration(cl);
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public Entry<K, V> randomEntry() {
-        throw new UnsupportedOperationException("Method should be supported.");
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteCache<K, V> withExpiryPolicy(ExpiryPolicy plc) {
-        throw new UnsupportedOperationException("Method should be supported.");
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteCache<K, V> withSkipStore() {
-        throw new UnsupportedOperationException("Method should be supported.");
-    }
-
-    /** {@inheritDoc} */
-    @Override public void loadCache(@Nullable IgniteBiPredicate<K, V> p, @Nullable Object... args) throws CacheException {
-        throw new UnsupportedOperationException("Method should be supported.");
-    }
-
-    /** {@inheritDoc} */
-    @Override public void localLoadCache(@Nullable final IgniteBiPredicate<K, V> p, @Nullable final Object... args) throws CacheException {
-        final IgniteBiPredicate pCopy = p;
-
-        compute.run(new IgniteRunnable() {
-            @Override public void run() {
-                cache().localLoadCache(pCopy, args);
-            }
-        });
-    }
-
-    /** {@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 cache().getAndPutIfAbsent(key, val);
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public Lock lock(K key) {
-        throw new UnsupportedOperationException("Method should be supported.");
-    }
-
-    /** {@inheritDoc} */
-    @Override public Lock lockAll(Collection<? extends K> keys) {
-        throw new UnsupportedOperationException("Method should be supported.");
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean isLocalLocked(K key, boolean byCurrThread) {
-        throw new UnsupportedOperationException("Method should be supported.");
-    }
-
-    /** {@inheritDoc} */
-    @Override public <R> QueryCursor<R> query(Query<R> qry) {
-        throw new UnsupportedOperationException("Method should be supported.");
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Override public Iterable<Entry<K, V>> localEntries(final CachePeekMode... peekModes) throws CacheException {
-        return (Iterable<Entry<K, V>>)compute.call(new IgniteCallable<Object>() {
-            @Override public Object call() throws Exception {
-                Collection<Entry> res = new ArrayList<>();
-
-                for (Entry e : cache().localEntries(peekModes))
-                    res.add(e);
-
-                return res;
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public QueryMetrics queryMetrics() {
-        throw new UnsupportedOperationException("Method should be supported.");
-    }
-
-    /** {@inheritDoc} */
-    @Override public void localEvict(final Collection<? extends K> keys) {
-        compute.run(new IgniteRunnable() {
-            @Override public void run() {
-                cache().localEvict(keys);
-            }
-        });
-    }
-
-    /** {@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 cache().localPeek(key, peekModes);
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public void localPromote(Set<? extends K> keys) throws CacheException {
-        throw new UnsupportedOperationException("Method should be supported.");
-    }
-
-    /** {@inheritDoc} */
-    @Override public int size(final CachePeekMode... peekModes) throws CacheException {
-        return (int)compute.call(new IgniteCallable<Object>() {
-            @Override public Object call() throws Exception {
-                return cache().size(peekModes);
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public int localSize(final CachePeekMode... peekModes) {
-        return (int)compute.call(new IgniteCallable<Object>() {
-            @Override public Object call() throws Exception {
-                return cache().localSize(peekModes);
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override  public <T> Map<K, EntryProcessorResult<T>> invokeAll(Map<? extends K, ? extends EntryProcessor<K, V, T>> map,
-        Object... args) {
-        throw new UnsupportedOperationException("Method should be supported.");
-    }
-
-    /** {@inheritDoc} */
-    @Override public V get(final K key) {
-        return (V)compute.call(new IgniteCallable<Object>() {
-            @Override public Object call() throws Exception {
-                return cache().get(key);
-            }
-        });
-    }
-
-    /**
-     * 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>() {
-            @Override public Object call() throws Exception {
-                return cache().getAll(keys);
-            }
-        });
-    }
-
-    @Override public Map<K, V> getAllOutTx(final Set<? extends K> keys) {
-        return (Map<K, V>)compute.call(new IgniteCallable<Object>() {
-            @Override public Object call() throws Exception {
-                return cache().getAllOutTx(keys);
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean containsKey(final K key) {
-        return (boolean)compute.call(new IgniteCallable<Object>() {
-            @Override public Object call() throws Exception {
-                return cache().containsKey(key);
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override  public void loadAll(Set<? extends K> keys, boolean replaceExistingValues, CompletionListener completionListener) {
-        throw new UnsupportedOperationException("Method should be supported.");
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean containsKeys(final Set<? extends K> keys) {
-        return (boolean)compute.call(new IgniteCallable<Object>() {
-            @Override public Object call() throws Exception {
-                return cache().containsKeys(keys);
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public void put(final K key, final V val) {;
-        compute.run(new IgniteRunnable() {
-            @Override public void run() {
-                cache().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 cache().getAndPut(key, val);
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public void putAll(final Map<? extends K, ? extends V> map) {
-        compute.run(new IgniteRunnable() {
-            @Override public void run() {
-                cache().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 cache().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 cache().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 cache().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 cache().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 cache().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 cache().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 cache().getAndReplace(key, val);
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public void removeAll(final Set<? extends K> keys) {
-        compute.run(new IgniteRunnable() {
-            @Override public void run() {
-                cache().removeAll(keys);
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public void removeAll() {
-        compute.run(new IgniteRunnable() {
-            @Override public void run() {
-                cache().removeAll();
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public void clear() {
-        compute.run(new IgniteRunnable() {
-            @Override public void run() {
-                cache().clear();
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public void clear(final K key) {
-        compute.run(new IgniteRunnable() {
-            @Override public void run() {
-                cache().clear(key);
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public void clearAll(final Set<? extends K> keys) {
-        compute.run(new IgniteRunnable() {
-            @Override public void run() {
-                cache().clearAll(keys);
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public void localClear(final K key) {
-        compute.run(new IgniteRunnable() {
-            @Override public void run() {
-                cache().localClear(key);
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public void localClearAll(final Set<? extends K> keys) {
-        compute.run(new IgniteRunnable() {
-            @Override public void run() {
-                cache().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 cache().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 cache().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 cache().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 cache().getName();
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public CacheManager getCacheManager() {
-        throw new UnsupportedOperationException("Method should be supported.");
-    }
-
-    /** {@inheritDoc} */
-    @Override public void close() {
-        compute.run(new IgniteRunnable() {
-            @Override public void run() {
-                cache().close();
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean isClosed() {
-        return (boolean)compute.call(new IgniteCallable<Object>() {
-            @Override public Object call() throws Exception {
-                return cache().isClosed();
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public <T> T unwrap(final Class<T> clazz) {
-        if (Ignite.class.equals(clazz))
-            return (T)igniteProxy;
-
-        try {
-            return (T)compute.call(new IgniteCallable<Object>() {
-                @Override public Object call() throws Exception {
-                    return cache().unwrap(clazz);
-                }
-            });
-        }
-        catch (Exception e) {
-            throw new IllegalArgumentException("Looks like class " + clazz + " is unmarshallable. Exception type:" + e.getClass(), e);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override  public void registerCacheEntryListener(CacheEntryListenerConfiguration<K, V> cacheEntryListenerConfiguration) {
-        throw new UnsupportedOperationException("Method should be supported.");
-    }
-
-    /** {@inheritDoc} */
-    @Override  public void deregisterCacheEntryListener(CacheEntryListenerConfiguration<K, V> cacheEntryListenerConfiguration) {
-        throw new UnsupportedOperationException("Method should be supported.");
-    }
-
-    /** {@inheritDoc} */
-    @Override public Iterator<Entry<K, V>> iterator() {
-        final Collection<Entry<K, V>> col = (Collection<Entry<K, V>>)compute.call(new IgniteCallable<Object>() {
-            @Override public Object call() throws Exception {
-                Collection res = new ArrayList();
-
-                for (Object o : cache())
-                    res.add(o);
-
-                return res;
-            }
-        });
-
-        return col.iterator();
-    }
-
-    /** {@inheritDoc} */
-    @Override public <T> Map<K, EntryProcessorResult<T>> invokeAll(Set<? extends K> keys,
-        CacheEntryProcessor<K, V, T> entryProcessor, Object... args) {
-        throw new UnsupportedOperationException("Method should be supported.");
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteFuture<?> rebalance() {
-        throw new UnsupportedOperationException("Method should be supported.");
-    }
-
-    /** {@inheritDoc} */
-    @Override public CacheMetrics metrics() {
-        throw new UnsupportedOperationException("Method should be supported.");
-    }
-
-    /** {@inheritDoc} */
-    @Override public CacheMetrics metrics(ClusterGroup grp) {
-        throw new UnsupportedOperationException("Method should be supported.");
-    }
-
-    /** {@inheritDoc} */
-    @Override public CacheMetricsMXBean mxBean() {
-        throw new UnsupportedOperationException("Method should be supported.");
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b77db74b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteEventsProcessProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteEventsProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteEventsProcessProxy.java
deleted file mode 100644
index b9a260b..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteEventsProcessProxy.java
+++ /dev/null
@@ -1,147 +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.framework;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cluster.*;
-import org.apache.ignite.events.*;
-import org.apache.ignite.lang.*;
-import org.jetbrains.annotations.*;
-
-import java.util.*;
-
-/**
- * Ignite events proxy for ignite instance at another JVM.
- */
-public class IgniteEventsProcessProxy implements IgniteEvents {
-    /** Ignite proxy. */
-    private final transient IgniteProcessProxy igniteProxy;
-
-    /** Grid id. */
-    private final UUID gridId;
-
-    /**
-     * @param igniteProxy Ignite proxy.
-     */
-    public IgniteEventsProcessProxy(IgniteProcessProxy igniteProxy) {
-        this.igniteProxy = igniteProxy;
-
-        gridId = igniteProxy.getId();
-    }
-
-    /**
-     * @return Events instance.
-     */
-    private IgniteEvents events() {
-        return Ignition.ignite(gridId).events();
-    }
-
-    /** {@inheritDoc} */
-    @Override public ClusterGroup clusterGroup() {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public <T extends Event> List<T> remoteQuery(IgnitePredicate<T> p, long timeout,
-        @Nullable int... types) throws IgniteException {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@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.
-    }
-
-    /** {@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.
-    }
-
-    /** {@inheritDoc} */
-    @Override public void stopRemoteListen(UUID opId) throws IgniteException {
-        // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public <T extends Event> T waitForLocal(@Nullable IgnitePredicate<T> filter,
-        @Nullable int... types) throws IgniteException {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public <T extends Event> Collection<T> localQuery(IgnitePredicate<T> p, @Nullable int... types) {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public void recordLocal(Event evt) {
-        // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public void localListen(final IgnitePredicate<? extends Event> lsnr, final int... types) {
-        igniteProxy.remoteCompute().run(new IgniteRunnable() {
-            @Override public void run() {
-                events().localListen(lsnr, types);
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean stopLocalListen(IgnitePredicate<? extends Event> lsnr, @Nullable int... types) {
-        return false; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public void enableLocal(int... types) {
-        // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public void disableLocal(int... types) {
-        // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public int[] enabledEvents() {
-        return new int[0]; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean isEnabled(int type) {
-        return false; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteEvents 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.
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b77db74b/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
deleted file mode 100644
index 493d515..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteNodeRunner.java
+++ /dev/null
@@ -1,111 +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.framework;
-
-import com.thoughtworks.xstream.*;
-import org.apache.ignite.*;
-import org.apache.ignite.configuration.*;
-import org.apache.ignite.internal.processors.cache.*;
-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 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 = readCfgFromFileAndDeleteFile(args[0]);
-
-            Ignition.start(cfg);
-        }
-        catch (Throwable e) {
-            e.printStackTrace();
-
-            System.exit(1);
-        }
-    }
-
-    /**
-     * Stores {@link IgniteConfiguration} to file as xml.
-     *
-     * @param cfg Ignite Configuration.
-     * @return A name of file where the configuration was stored.
-     * @throws IOException If failed.
-     * @see #readCfgFromFileAndDeleteFile(String)
-     */
-    public static String storeToFile(IgniteConfiguration cfg) throws IOException {
-        String fileName = IGNITE_CONFIGURATION_FILE + cfg.getNodeId();
-
-        try(OutputStream out = new BufferedOutputStream(new FileOutputStream(fileName))) {
-            cfg.setMBeanServer(null);
-            cfg.setMarshaller(null);
-            cfg.setDiscoverySpi(null);
-            cfg.setGridLogger(null);
-
-            new XStream().toXML(cfg, out);
-        }
-
-        return fileName;
-    }
-
-    /**
-     * Reads configuration from given file and delete the file after.
-     *
-     * @param fileName File name.
-     * @return Readed configuration.
-     * @throws IOException If failed.
-     * @see #storeToFile(IgniteConfiguration)
-     */
-    private static IgniteConfiguration readCfgFromFileAndDeleteFile(String fileName) throws IOException {
-        try(BufferedReader cfgReader = new BufferedReader(new FileReader(fileName))) {
-            IgniteConfiguration cfg = (IgniteConfiguration)new XStream().fromXML(cfgReader);
-
-            cfg.setMarshaller(new OptimizedMarshaller(false));
-
-            TcpDiscoverySpi disco = new TcpDiscoverySpi();
-            disco.setIpFinder(GridCacheAbstractFullApiSelfTest.LOCAL_IP_FINDER);
-            cfg.setDiscoverySpi(disco);
-
-            return cfg;
-        }
-        finally {
-            new File(fileName).delete();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b77db74b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteProcessProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteProcessProxy.java
deleted file mode 100644
index e2751e1..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteProcessProxy.java
+++ /dev/null
@@ -1,467 +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.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.events.*;
-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.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 IgniteProcessProxy implements IgniteEx {
-    /** Grid proxies. */
-    private transient static final Map<String, IgniteProcessProxy> 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();
-
-    /** Remote ignite instance started latch. */
-    private transient final CountDownLatch rmtNodeStartedLatch = new CountDownLatch(1);
-
-    /**
-     * @param cfg Configuration.
-     * @param log Logger.
-     * @param locJvmGrid Local jvm grid.
-     */
-    public IgniteProcessProxy(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('-')));
-
-        String cfgFileName = 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);
-        }
-
-        locJvmGrid.events().localListen(new IgnitePredicateX<Event>() {
-            @Override public boolean applyx(Event e) {
-                if (((DiscoveryEvent)e).eventNode().id().equals(id)) {
-                    rmtNodeStartedLatch.countDown();
-
-                    return false;
-                }
-
-                return true;
-            }
-        }, EventType.EVT_NODE_JOINED);
-
-        proc = GridJavaProcess.exec(
-            IgniteNodeRunner.class,
-            cfgFileName, // 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) {
-                    IgniteProcessProxy.this.log.info(s);
-                }
-            },
-            null,
-            filteredJvmArgs, // JVM Args.
-            System.getProperty("surefire.test.class.path")
-        );
-
-        assert rmtNodeStartedLatch.await(30, TimeUnit.SECONDS): "Remote node with id=" + id + " didn't join.";
-
-        gridProxies.put(cfg.getGridName(), this);
-    }
-
-    /**
-     * @param gridName Grid name.
-     * @return Instance by name or exception wiil be thrown.
-     */
-    public static IgniteProcessProxy get(String gridName) {
-        IgniteProcessProxy res = gridProxies.get(gridName);
-
-        if (res == null)
-            throw new IgniteIllegalStateException("Grid instance was not properly started " +
-                "or was already stopped: " + gridName);
-
-        return res;
-    }
-
-    /**
-     * Kill all running processes.
-     */
-    public static void killAll() {
-        for (IgniteProcessProxy 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 remoteCompute().call(new IgniteCallable<ClusterNode>() {
-            @Override public ClusterNode call() throws Exception {
-                return ((IgniteEx)Ignition.ignite(id)).localNode();
-            }
-        });
-    }
-
-    /** {@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 new IgniteEventsProcessProxy(this);
-    }
-
-    /** {@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() {
-        throw new UnsupportedOperationException("Transactions are not supported in multi JVM mode.");
-    }
-
-    /** {@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 new AffinityProcessProxy(cacheName, this);
-    }
-
-    /**
-     * @return Jvm process in which grid node started.
-     */
-    public GridJavaProcess getProcess() {
-        return proc;
-    }
-
-    /**
-     * @return {@link IgniteCompute} instance to communicate with remote node.
-     */
-    public IgniteCompute remoteCompute() {
-        ClusterGroup grp = locJvmGrid.cluster().forNodeId(id);
-
-        if (grp.nodes().isEmpty())
-            throw new IllegalStateException("Could not found node with id=" + id + ".");
-
-        return locJvmGrid.compute(grp);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b77db74b/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 b67f5b8..da33693 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
@@ -23,7 +23,6 @@ import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.events.*;
 import org.apache.ignite.internal.*;
-import org.apache.ignite.internal.processors.cache.multijvm.framework.*;
 import org.apache.ignite.internal.processors.resource.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.internal.util.typedef.*;
@@ -40,6 +39,7 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
 import org.apache.ignite.testframework.*;
 import org.apache.ignite.testframework.config.*;
 import org.apache.ignite.testframework.junits.logger.*;
+import org.apache.ignite.testframework.junits.multijvm.*;
 import org.apache.log4j.*;
 import org.jetbrains.annotations.*;
 import org.springframework.beans.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b77db74b/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
new file mode 100644
index 0000000..2b699b6
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/AffinityProcessProxy.java
@@ -0,0 +1,194 @@
+/*
+ * 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.testframework.junits.multijvm;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.affinity.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.lang.*;
+import org.jetbrains.annotations.*;
+
+import java.util.*;
+
+/**
+ * Proxy class for affinity at another jvm.
+ */
+public class AffinityProcessProxy<K> implements Affinity<K> {
+    /** Compute. */
+    private final transient IgniteCompute compute;
+
+    /** Cache name. */
+    private final String cacheName;
+
+    /** Grid id. */
+    private final UUID gridId;
+
+    /**
+     * @param cacheName Cache name.
+     * @param proxy Ignite ptocess proxy.
+     */
+    public AffinityProcessProxy(String cacheName, IgniteProcessProxy proxy) {
+        this.cacheName = cacheName;
+        gridId = proxy.getId();
+        compute = proxy.remoteCompute();
+    }
+
+    /**
+     * Returns cache instance. Method to be called from closure at another JVM.
+     *
+     * @return Cache.
+     */
+    private Affinity<Object> affinity() {
+        return Ignition.ignite(gridId).affinity(cacheName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int partitions() {
+        return (int)compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                return affinity().partitions();
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public int partition(final K key) {
+        return (int)compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                return affinity().partition(key);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isPrimary(final ClusterNode n, final K key) {
+        return (boolean)compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                return affinity().isPrimary(n, key);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isBackup(final ClusterNode n, final K key) {
+        return (boolean)compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                return affinity().isBackup(n, key);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isPrimaryOrBackup(final ClusterNode n, final K key) {
+        return (boolean)compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                return affinity().isPrimaryOrBackup(n, key);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public int[] primaryPartitions(final ClusterNode n) {
+        return (int[])compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                return affinity().primaryPartitions(n);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public int[] backupPartitions(final ClusterNode n) {
+        return (int[])compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                return affinity().backupPartitions(n);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public int[] allPartitions(final ClusterNode n) {
+        return (int[])compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                return affinity().allPartitions(n);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object affinityKey(final K key) {
+        return compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                return affinity().affinityKey(key);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public Map<ClusterNode, Collection<K>> mapKeysToNodes(final Collection<? extends K> keys) {
+        return (Map<ClusterNode, Collection<K>>)compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                return affinity().mapKeysToNodes(keys);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public ClusterNode mapKeyToNode(final K key) {
+        return (ClusterNode)compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                return affinity().mapKeyToNode(key);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<ClusterNode> mapKeyToPrimaryAndBackups(final K key) {
+        return (Collection<ClusterNode>)compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                return affinity().mapKeyToPrimaryAndBackups(key);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterNode mapPartitionToNode(final int part) {
+        return (ClusterNode)compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                return affinity().mapPartitionToNode(part);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public Map<Integer, ClusterNode> mapPartitionsToNodes(final Collection<Integer> parts) {
+        return (Map<Integer, ClusterNode>)compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                return affinity().mapPartitionsToNodes(parts);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<ClusterNode> mapPartitionToPrimaryAndBackups(final int part) {
+        return (Collection<ClusterNode>)compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                return affinity().mapPartitionToPrimaryAndBackups(part);
+            }
+        });
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b77db74b/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
new file mode 100644
index 0000000..d4dccab
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteCacheProcessProxy.java
@@ -0,0 +1,586 @@
+/*
+ * 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.testframework.junits.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;
+
+    /** With async. */
+    private final boolean isAsync;
+
+    /** Ignite proxy. */
+    private final transient IgniteProcessProxy igniteProxy;
+
+    /**
+     * @param name Name.
+     * @param proxy Ignite Process Proxy.
+     */
+    public IgniteCacheProcessProxy(String name, IgniteProcessProxy proxy) {
+        this(name, false, proxy);
+    }
+
+    /**
+     * @param name Name.
+     * @param async
+     * @param proxy Ignite Process Proxy.
+     */
+    public IgniteCacheProcessProxy(String name, boolean async, IgniteProcessProxy proxy) {
+        cacheName = name;
+        isAsync = async;
+        gridId = proxy.getId();
+        igniteProxy = proxy;
+        compute = proxy.remoteCompute();
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteCache<K, V> withAsync() {
+        return new IgniteCacheProcessProxy<>(cacheName, true, igniteProxy);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isAsync() {
+        return isAsync;
+    }
+
+    /** {@inheritDoc} */
+    @Override public <R> IgniteFuture<R> future() {
+        // TODO implement.
+//        R futureRes = (R)compute.call(new IgniteCallable<Object>() {
+//            @Override public Object call() throws Exception {
+//                return cache().future().get();
+//            }
+//        });
+//
+//        return new IgniteFinishedFutureImpl<R>(futureRes);
+
+        throw new UnsupportedOperationException("Method should be supported.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public <C extends Configuration<K, V>> C getConfiguration(final Class<C> clazz) {
+        final Class cl = clazz;
+
+        return (C)compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                return cache().getConfiguration(cl);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public Entry<K, V> randomEntry() {
+        throw new UnsupportedOperationException("Method should be supported.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteCache<K, V> withExpiryPolicy(ExpiryPolicy plc) {
+        throw new UnsupportedOperationException("Method should be supported.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteCache<K, V> withSkipStore() {
+        throw new UnsupportedOperationException("Method should be supported.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void loadCache(@Nullable IgniteBiPredicate<K, V> p, @Nullable Object... args) throws CacheException {
+        throw new UnsupportedOperationException("Method should be supported.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void localLoadCache(@Nullable final IgniteBiPredicate<K, V> p, @Nullable final Object... args) throws CacheException {
+        final IgniteBiPredicate pCopy = p;
+
+        compute.run(new IgniteRunnable() {
+            @Override public void run() {
+                cache().localLoadCache(pCopy, args);
+            }
+        });
+    }
+
+    /** {@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 cache().getAndPutIfAbsent(key, val);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public Lock lock(K key) {
+        throw new UnsupportedOperationException("Method should be supported.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public Lock lockAll(Collection<? extends K> keys) {
+        throw new UnsupportedOperationException("Method should be supported.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isLocalLocked(K key, boolean byCurrThread) {
+        throw new UnsupportedOperationException("Method should be supported.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public <R> QueryCursor<R> query(Query<R> qry) {
+        throw new UnsupportedOperationException("Method should be supported.");
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public Iterable<Entry<K, V>> localEntries(final CachePeekMode... peekModes) throws CacheException {
+        return (Iterable<Entry<K, V>>)compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                Collection<Entry> res = new ArrayList<>();
+
+                for (Entry e : cache().localEntries(peekModes))
+                    res.add(e);
+
+                return res;
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public QueryMetrics queryMetrics() {
+        throw new UnsupportedOperationException("Method should be supported.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void localEvict(final Collection<? extends K> keys) {
+        compute.run(new IgniteRunnable() {
+            @Override public void run() {
+                cache().localEvict(keys);
+            }
+        });
+    }
+
+    /** {@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 cache().localPeek(key, peekModes);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public void localPromote(Set<? extends K> keys) throws CacheException {
+        throw new UnsupportedOperationException("Method should be supported.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public int size(final CachePeekMode... peekModes) throws CacheException {
+        return (int)compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                return cache().size(peekModes);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public int localSize(final CachePeekMode... peekModes) {
+        return (int)compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                return cache().localSize(peekModes);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override  public <T> Map<K, EntryProcessorResult<T>> invokeAll(Map<? extends K, ? extends EntryProcessor<K, V, T>> map,
+        Object... args) {
+        throw new UnsupportedOperationException("Method should be supported.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public V get(final K key) {
+        return (V)compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                return cache().get(key);
+            }
+        });
+    }
+
+    /**
+     * 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>() {
+            @Override public Object call() throws Exception {
+                return cache().getAll(keys);
+            }
+        });
+    }
+
+    @Override public Map<K, V> getAllOutTx(final Set<? extends K> keys) {
+        return (Map<K, V>)compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                return cache().getAllOutTx(keys);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean containsKey(final K key) {
+        return (boolean)compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                return cache().containsKey(key);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override  public void loadAll(Set<? extends K> keys, boolean replaceExistingValues, CompletionListener completionListener) {
+        throw new UnsupportedOperationException("Method should be supported.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean containsKeys(final Set<? extends K> keys) {
+        return (boolean)compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                return cache().containsKeys(keys);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public void put(final K key, final V val) {;
+        compute.run(new IgniteRunnable() {
+            @Override public void run() {
+                cache().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 cache().getAndPut(key, val);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public void putAll(final Map<? extends K, ? extends V> map) {
+        compute.run(new IgniteRunnable() {
+            @Override public void run() {
+                cache().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 cache().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 cache().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 cache().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 cache().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 cache().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 cache().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 cache().getAndReplace(key, val);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public void removeAll(final Set<? extends K> keys) {
+        compute.run(new IgniteRunnable() {
+            @Override public void run() {
+                cache().removeAll(keys);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public void removeAll() {
+        compute.run(new IgniteRunnable() {
+            @Override public void run() {
+                cache().removeAll();
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public void clear() {
+        compute.run(new IgniteRunnable() {
+            @Override public void run() {
+                cache().clear();
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public void clear(final K key) {
+        compute.run(new IgniteRunnable() {
+            @Override public void run() {
+                cache().clear(key);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public void clearAll(final Set<? extends K> keys) {
+        compute.run(new IgniteRunnable() {
+            @Override public void run() {
+                cache().clearAll(keys);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public void localClear(final K key) {
+        compute.run(new IgniteRunnable() {
+            @Override public void run() {
+                cache().localClear(key);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public void localClearAll(final Set<? extends K> keys) {
+        compute.run(new IgniteRunnable() {
+            @Override public void run() {
+                cache().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 cache().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 cache().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 cache().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 cache().getName();
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public CacheManager getCacheManager() {
+        throw new UnsupportedOperationException("Method should be supported.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void close() {
+        compute.run(new IgniteRunnable() {
+            @Override public void run() {
+                cache().close();
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isClosed() {
+        return (boolean)compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                return cache().isClosed();
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> T unwrap(final Class<T> clazz) {
+        if (Ignite.class.equals(clazz))
+            return (T)igniteProxy;
+
+        try {
+            return (T)compute.call(new IgniteCallable<Object>() {
+                @Override public Object call() throws Exception {
+                    return cache().unwrap(clazz);
+                }
+            });
+        }
+        catch (Exception e) {
+            throw new IllegalArgumentException("Looks like class " + clazz + " is unmarshallable. Exception type:" + e.getClass(), e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override  public void registerCacheEntryListener(CacheEntryListenerConfiguration<K, V> cacheEntryListenerConfiguration) {
+        throw new UnsupportedOperationException("Method should be supported.");
+    }
+
+    /** {@inheritDoc} */
+    @Override  public void deregisterCacheEntryListener(CacheEntryListenerConfiguration<K, V> cacheEntryListenerConfiguration) {
+        throw new UnsupportedOperationException("Method should be supported.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public Iterator<Entry<K, V>> iterator() {
+        final Collection<Entry<K, V>> col = (Collection<Entry<K, V>>)compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                Collection res = new ArrayList();
+
+                for (Object o : cache())
+                    res.add(o);
+
+                return res;
+            }
+        });
+
+        return col.iterator();
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> Map<K, EntryProcessorResult<T>> invokeAll(Set<? extends K> keys,
+        CacheEntryProcessor<K, V, T> entryProcessor, Object... args) {
+        throw new UnsupportedOperationException("Method should be supported.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteFuture<?> rebalance() {
+        throw new UnsupportedOperationException("Method should be supported.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public CacheMetrics metrics() {
+        throw new UnsupportedOperationException("Method should be supported.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public CacheMetrics metrics(ClusterGroup grp) {
+        throw new UnsupportedOperationException("Method should be supported.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public CacheMetricsMXBean mxBean() {
+        throw new UnsupportedOperationException("Method should be supported.");
+    }
+}