You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ag...@apache.org on 2017/08/01 09:27:06 UTC

[21/50] [abbrv] ignite git commit: IGNITE-5770 Refactor PlatformProcessor to PlatformTarget mechanism

IGNITE-5770 Refactor PlatformProcessor to PlatformTarget mechanism

This closes #2336


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

Branch: refs/heads/ignite-5757
Commit: bdc9e4b91403595c496a5fa4aca70228665feaaa
Parents: bd7d407
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Tue Jul 25 15:15:43 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Tue Jul 25 15:15:43 2017 +0300

----------------------------------------------------------------------
 .../platform/PlatformAbstractTarget.java        |   6 +-
 .../processors/platform/PlatformIgnition.java   |   5 +-
 .../platform/PlatformNoopProcessor.java         | 132 -----
 .../processors/platform/PlatformProcessor.java  | 216 --------
 .../platform/PlatformProcessorImpl.java         | 554 +++++++++++--------
 .../platform/cluster/PlatformClusterGroup.java  |  29 +
 .../datastructures/PlatformAtomicReference.java |  13 +-
 .../platforms/cpp/core/include/ignite/ignite.h  |   6 +-
 .../include/ignite/impl/compute/compute_impl.h  |   5 +-
 .../cpp/core/include/ignite/impl/ignite_impl.h  | 112 ++--
 .../ignite/impl/interop/interop_target.h        |  16 +-
 modules/platforms/cpp/core/src/ignition.cpp     |  23 +-
 .../cpp/core/src/impl/ignite_environment.cpp    |  37 +-
 .../platforms/cpp/core/src/impl/ignite_impl.cpp |  25 +-
 .../core/src/impl/interop/interop_target.cpp    |  18 +-
 .../cpp/jni/include/ignite/jni/exports.h        |  31 +-
 .../platforms/cpp/jni/include/ignite/jni/java.h |  70 +--
 modules/platforms/cpp/jni/project/vs/module.def |  30 +-
 modules/platforms/cpp/jni/src/exports.cpp       | 116 +---
 modules/platforms/cpp/jni/src/java.cpp          | 518 +----------------
 .../dotnet/Apache.Ignite.Core/Ignition.cs       |  12 +-
 .../Apache.Ignite.Core/Impl/Cache/CacheImpl.cs  |   2 +-
 .../Impl/Cluster/ClusterGroupImpl.cs            |  78 ++-
 .../Impl/Compute/ComputeImpl.cs                 |   6 +-
 .../Impl/Datastream/DataStreamerImpl.cs         |   3 +-
 .../Impl/Datastream/StreamReceiverHolder.cs     |   2 +-
 .../dotnet/Apache.Ignite.Core/Impl/Ignite.cs    | 279 +++++-----
 .../Apache.Ignite.Core/Impl/Log/JavaLogger.cs   |  23 +-
 .../Apache.Ignite.Core/Impl/PlatformTarget.cs   |  18 +-
 .../Impl/Plugin/PluginContext.cs                |   7 +-
 .../Impl/Unmanaged/IgniteJniNativeMethods.cs    |  91 +--
 .../Impl/Unmanaged/UnmanagedUtils.cs            | 308 +----------
 32 files changed, 768 insertions(+), 2023 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/bdc9e4b9/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java
index 5e785e2..4a584f9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java
@@ -130,7 +130,7 @@ public abstract class PlatformAbstractTarget implements PlatformTarget {
      * @return Dummy value which is never returned.
      * @throws IgniteCheckedException Exception to be thrown.
      */
-    private <T> T throwUnsupported(int type) throws IgniteCheckedException {
+    public static <T> T throwUnsupported(int type) throws IgniteCheckedException {
         throw new IgniteCheckedException("Unsupported operation type: " + type);
     }
 
@@ -142,8 +142,8 @@ public abstract class PlatformAbstractTarget implements PlatformTarget {
      * @param writer Writer.
      * @throws IgniteCheckedException In case of error.
      */
-    protected PlatformListenable readAndListenFuture(BinaryRawReader reader, IgniteInternalFuture fut,
-                                                     PlatformFutureUtils.Writer writer)
+    private PlatformListenable readAndListenFuture(BinaryRawReader reader, IgniteInternalFuture fut,
+                                                   PlatformFutureUtils.Writer writer)
             throws IgniteCheckedException {
         long futId = reader.readLong();
         int futTyp = reader.readInt();

http://git-wip-us.apache.org/repos/asf/ignite/blob/bdc9e4b9/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformIgnition.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformIgnition.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformIgnition.java
index 422e16e..754c69e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformIgnition.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformIgnition.java
@@ -49,9 +49,8 @@ public class PlatformIgnition {
      * @param factoryId Factory ID.
      * @param envPtr Environment pointer.
      * @param dataPtr Optional pointer to additional data required for startup.
-     * @return Ignite instance.
      */
-    public static synchronized PlatformProcessor start(@Nullable String springCfgPath,
+    public static synchronized void start(@Nullable String springCfgPath,
         @Nullable String igniteInstanceName, int factoryId, long envPtr, long dataPtr) {
         if (envPtr <= 0)
             throw new IgniteException("Environment pointer must be positive.");
@@ -78,8 +77,6 @@ public class PlatformIgnition {
             PlatformProcessor old = instances.put(igniteInstanceName, proc);
 
             assert old == null;
-
-            return proc;
         }
         finally {
             Thread.currentThread().setContextClassLoader(oldClsLdr);

http://git-wip-us.apache.org/repos/asf/ignite/blob/bdc9e4b9/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java
index cd170ed..f26826e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java
@@ -23,7 +23,6 @@ import org.apache.ignite.IgniteException;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.processors.GridProcessorAdapter;
 import org.apache.ignite.internal.processors.platform.cache.store.PlatformCacheStore;
-import org.jetbrains.annotations.Nullable;
 
 /**
  * No-op processor.
@@ -61,139 +60,8 @@ public class PlatformNoopProcessor extends GridProcessorAdapter implements Platf
     }
 
     /** {@inheritDoc} */
-    @Override public PlatformTargetProxy cache(@Nullable String name) throws IgniteCheckedException {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public PlatformTargetProxy createCache(@Nullable String name) throws IgniteCheckedException {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public PlatformTargetProxy getOrCreateCache(@Nullable String name) throws IgniteCheckedException {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public PlatformTargetProxy createCacheFromConfig(long memPtr) throws IgniteCheckedException {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public PlatformTargetProxy getOrCreateCacheFromConfig(long memPtr) throws IgniteCheckedException {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void destroyCache(@Nullable String name) throws IgniteCheckedException {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public PlatformTargetProxy affinity(@Nullable String name) throws IgniteCheckedException {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public PlatformTargetProxy dataStreamer(@Nullable String cacheName, boolean keepBinary)
-        throws IgniteCheckedException {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public PlatformTargetProxy transactions() {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public PlatformTargetProxy projection() throws IgniteCheckedException {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public PlatformTargetProxy compute(PlatformTargetProxy grp) {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public PlatformTargetProxy message(PlatformTargetProxy grp) {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public PlatformTargetProxy events(PlatformTargetProxy grp) {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public PlatformTargetProxy services(PlatformTargetProxy grp) {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public PlatformTargetProxy extensions() {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public PlatformTargetProxy extension(int id) {
-        return null;
-    }
-
-    /** {@inheritDoc} */
     @Override public void registerStore(PlatformCacheStore store, boolean convertBinary)
         throws IgniteCheckedException {
         // No-op.
     }
-
-    /** {@inheritDoc} */
-    @Override public PlatformTargetProxy atomicLong(String name, long initVal, boolean create) throws IgniteException {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void getIgniteConfiguration(long memPtr) {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public void getCacheNames(long memPtr) {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public PlatformTargetProxy atomicSequence(String name, long initVal, boolean create) throws IgniteException {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public PlatformTargetProxy atomicReference(String name, long memPtr, boolean create) throws IgniteException {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public PlatformTargetProxy createNearCache(@Nullable String cacheName, long memPtr) {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public PlatformTargetProxy getOrCreateNearCache(@Nullable String cacheName, long memPtr) {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean loggerIsLevelEnabled(int level) {
-        return false;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void loggerLog(int level, String message, String category, String errorInfo) {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public PlatformTargetProxy binaryProcessor() {
-        return null;
-    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/bdc9e4b9/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java
index 54f33a7..9c17d78 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java
@@ -62,139 +62,6 @@ public interface PlatformProcessor extends GridProcessor {
     public void awaitStart() throws IgniteCheckedException;
 
     /**
-     * Get cache.
-     *
-     * @param name Cache name.
-     * @return Cache.
-     * @throws IgniteCheckedException If failed.
-     */
-    public PlatformTargetProxy cache(@Nullable String name) throws IgniteCheckedException;
-
-    /**
-     * Create cache.
-     *
-     * @param name Cache name.
-     * @return Cache.
-     * @throws IgniteCheckedException If failed.
-     */
-    public PlatformTargetProxy createCache(@Nullable String name) throws IgniteCheckedException;
-
-    /**
-     * Get or create cache.
-     *
-     * @param name Cache name.
-     * @return Cache.
-     * @throws IgniteCheckedException If failed.
-     */
-    public PlatformTargetProxy getOrCreateCache(@Nullable String name) throws IgniteCheckedException;
-
-    /**
-     * Create cache.
-     *
-     * @param memPtr Stream with cache config.
-     * @return Cache.
-     * @throws IgniteCheckedException If failed.
-     */
-    public PlatformTargetProxy createCacheFromConfig(long memPtr) throws IgniteCheckedException;
-
-    /**
-     * Get or create cache.
-     *
-     * @param memPtr Stream with cache config.
-     * @return Cache.
-     * @throws IgniteCheckedException If failed.
-     */
-    public PlatformTargetProxy getOrCreateCacheFromConfig(long memPtr) throws IgniteCheckedException;
-
-    /**
-     * Destroy dynamically created cache.
-     *
-     * @param name Cache name.
-     * @throws IgniteCheckedException If failed.
-     */
-    public void destroyCache(@Nullable String name) throws IgniteCheckedException;
-
-    /**
-     * Get affinity.
-     *
-     * @param name Cache name.
-     * @return Affinity.
-     * @throws IgniteCheckedException If failed.
-     */
-    public PlatformTargetProxy affinity(@Nullable String name) throws IgniteCheckedException;
-
-    /**
-     * Get data streamer.
-     *
-     * @param cacheName Cache name.
-     * @param keepBinary Binary flag.
-     * @return Data streamer.
-     * @throws IgniteCheckedException If failed.
-     */
-    public PlatformTargetProxy dataStreamer(@Nullable String cacheName, boolean keepBinary) throws IgniteCheckedException;
-
-    /**
-     * Get transactions.
-     *
-     * @return Transactions.
-     */
-    public PlatformTargetProxy transactions();
-
-    /**
-     * Get projection.
-     *
-     * @return Projection.
-     * @throws IgniteCheckedException If failed.
-     */
-    public PlatformTargetProxy projection() throws IgniteCheckedException;
-
-    /**
-     * Create interop compute.
-     *
-     * @param grp Cluster group.
-     * @return Compute instance.
-     */
-    public PlatformTargetProxy compute(PlatformTargetProxy grp);
-
-    /**
-     * Create interop messaging.
-     *
-     * @param grp Cluster group.
-     * @return Messaging instance.
-     */
-    public PlatformTargetProxy message(PlatformTargetProxy grp);
-
-    /**
-     * Create interop events.
-     *
-     * @param grp Cluster group.
-     * @return Events instance.
-     */
-    public PlatformTargetProxy events(PlatformTargetProxy grp);
-
-    /**
-     * Create interop services.
-     *
-     * @param grp Cluster group.
-     * @return Services instance.
-     */
-    public PlatformTargetProxy services(PlatformTargetProxy grp);
-
-    /**
-     * Get platform extensions. Override this method to provide any additional targets and operations you need.
-     *
-     * @return Platform extensions.
-     */
-    public PlatformTargetProxy extensions();
-
-    /**
-     * Gets platform extension by id.
-     *
-     * @return Platform extension target.
-     */
-    public PlatformTargetProxy extension(int id);
-
-    /**
      * Register cache store.
      *
      * @param store Store.
@@ -202,87 +69,4 @@ public interface PlatformProcessor extends GridProcessor {
      * @throws IgniteCheckedException If failed.
      */
     public void registerStore(PlatformCacheStore store, boolean convertBinary) throws IgniteCheckedException;
-
-    /**
-     * Get or create AtomicLong.
-     * @param name Name.
-     * @param initVal Initial value.
-     * @param create Create flag.
-     * @return Platform atomic long.
-     */
-    public PlatformTargetProxy atomicLong(String name, long initVal, boolean create);
-
-    /**
-     * Get or create AtomicSequence.
-     * @param name Name.
-     * @param initVal Initial value.
-     * @param create Create flag.
-     * @return Platform atomic long.
-     */
-    public PlatformTargetProxy atomicSequence(String name, long initVal, boolean create);
-
-    /**
-     * Get or create AtomicReference.
-     * @param name Name.
-     * @param memPtr Pointer to a stream with initial value. 0 for null initial value.
-     * @param create Create flag.
-     * @return Platform atomic long.
-     */
-    public PlatformTargetProxy atomicReference(String name, long memPtr, boolean create);
-
-    /**
-     * Gets the configuration of the current Ignite instance.
-     *
-     * @param memPtr Stream to write data to.
-     */
-    public void getIgniteConfiguration(long memPtr);
-
-    /**
-     * Gets the cache names.
-     *
-     * @param memPtr Stream to write data to.
-     */
-    public void getCacheNames(long memPtr);
-
-    /**
-     * Starts a near cache on local node if cache was previously started.
-     *
-     * @param cacheName Cache name.
-     * @param memPtr Pointer to a stream with near cache config. 0 for default config.
-     * @return Cache.
-     */
-    public PlatformTargetProxy createNearCache(@Nullable String cacheName, long memPtr);
-
-    /**
-     * Gets existing near cache with the given name or creates a new one.
-     *
-     * @param cacheName Cache name.
-     * @param memPtr Pointer to a stream with near cache config. 0 for default config.
-     * @return Cache.
-     */
-    public PlatformTargetProxy getOrCreateNearCache(@Nullable String cacheName, long memPtr);
-
-    /**
-     * Gets a value indicating whether Ignite logger has specified level enabled.
-     *
-     * @param level Log level.
-     */
-    public boolean loggerIsLevelEnabled(int level);
-
-    /**
-     * Logs to the Ignite logger.
-     *
-     * @param level Log level.
-     * @param message Message.
-     * @param category Category.
-     * @param errorInfo Error info.
-     */
-    public void loggerLog(int level, String message, String category, String errorInfo);
-
-    /**
-     * Gets the binary processor.
-     *
-     * @return Binary processor.
-     */
-    public PlatformTargetProxy binaryProcessor();
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/bdc9e4b9/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java
index 1da3112..612f154 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java
@@ -41,17 +41,13 @@ import org.apache.ignite.internal.processors.platform.cache.PlatformCacheExtensi
 import org.apache.ignite.internal.processors.platform.cache.affinity.PlatformAffinity;
 import org.apache.ignite.internal.processors.platform.cache.store.PlatformCacheStore;
 import org.apache.ignite.internal.processors.platform.cluster.PlatformClusterGroup;
-import org.apache.ignite.internal.processors.platform.compute.PlatformCompute;
 import org.apache.ignite.internal.processors.platform.datastreamer.PlatformDataStreamer;
 import org.apache.ignite.internal.processors.platform.datastructures.PlatformAtomicLong;
 import org.apache.ignite.internal.processors.platform.datastructures.PlatformAtomicReference;
 import org.apache.ignite.internal.processors.platform.datastructures.PlatformAtomicSequence;
 import org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore;
-import org.apache.ignite.internal.processors.platform.events.PlatformEvents;
 import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
 import org.apache.ignite.internal.processors.platform.memory.PlatformOutputStream;
-import org.apache.ignite.internal.processors.platform.messaging.PlatformMessaging;
-import org.apache.ignite.internal.processors.platform.services.PlatformServices;
 import org.apache.ignite.internal.processors.platform.transactions.PlatformTransactions;
 import org.apache.ignite.internal.processors.platform.utils.PlatformConfigurationUtils;
 import org.apache.ignite.internal.processors.platform.utils.PlatformUtils;
@@ -74,7 +70,73 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
  * GridGain platform processor.
  */
 @SuppressWarnings({"ConditionalExpressionWithIdenticalBranches", "unchecked"})
-public class PlatformProcessorImpl extends GridProcessorAdapter implements PlatformProcessor {
+public class PlatformProcessorImpl extends GridProcessorAdapter implements PlatformProcessor, PlatformTarget {
+    /** */
+    private static final int OP_GET_CACHE = 1;
+
+    /** */
+    private static final int OP_CREATE_CACHE = 2;
+
+    /** */
+    private static final int OP_GET_OR_CREATE_CACHE = 3;
+
+    /** */
+    private static final int OP_CREATE_CACHE_FROM_CONFIG = 4;
+
+    /** */
+    private static final int OP_GET_OR_CREATE_CACHE_FROM_CONFIG = 5;
+
+    /** */
+    private static final int OP_DESTROY_CACHE = 6;
+
+    /** */
+    private static final int OP_GET_AFFINITY = 7;
+
+    /** */
+    private static final int OP_GET_DATA_STREAMER = 8;
+
+    /** */
+    private static final int OP_GET_TRANSACTIONS = 9;
+
+    /** */
+    private static final int OP_GET_CLUSTER_GROUP = 10;
+
+    /** */
+    private static final int OP_GET_EXTENSION = 11;
+
+    /** */
+    private static final int OP_GET_ATOMIC_LONG = 12;
+
+    /** */
+    private static final int OP_GET_ATOMIC_REFERENCE = 13;
+
+    /** */
+    private static final int OP_GET_ATOMIC_SEQUENCE = 14;
+
+    /** */
+    private static final int OP_GET_IGNITE_CONFIGURATION = 15;
+
+    /** */
+    private static final int OP_GET_CACHE_NAMES = 16;
+
+    /** */
+    private static final int OP_CREATE_NEAR_CACHE = 17;
+
+    /** */
+    private static final int OP_GET_OR_CREATE_NEAR_CACHE = 18;
+
+    /** */
+    private static final int OP_LOGGER_IS_LEVEL_ENABLED = 19;
+
+    /** */
+    private static final int OP_LOGGER_LOG = 20;
+
+    /** */
+    private static final int OP_GET_BINARY_PROCESSOR = 21;
+
+    /** */
+    private static final int OP_RELEASE_START = 22;
+
     /** Start latch. */
     private final CountDownLatch startLatch = new CountDownLatch(1);
 
@@ -157,7 +219,7 @@ public class PlatformProcessorImpl extends GridProcessorAdapter implements Platf
 
             out.synchronize();
 
-            platformCtx.gateway().onStart(this, mem.pointer());
+            platformCtx.gateway().onStart(new PlatformTargetProxyImpl(this, platformCtx), mem.pointer());
         }
 
         // At this moment all necessary native libraries must be loaded, so we can process with store creation.
@@ -218,321 +280,360 @@ public class PlatformProcessorImpl extends GridProcessorAdapter implements Platf
     }
 
     /** {@inheritDoc} */
-    @Override public PlatformTargetProxy cache(@Nullable String name) throws IgniteCheckedException {
-        IgniteCacheProxy cache = (IgniteCacheProxy)ctx.grid().cache(name);
+    @Override public void registerStore(PlatformCacheStore store, boolean convertBinary)
+        throws IgniteCheckedException {
+        storeLock.readLock().lock();
 
-        if (cache == null)
-            throw new IllegalArgumentException("Cache doesn't exist: " + name);
+        try {
+            if (stopped)
+                throw new IgniteCheckedException("Failed to initialize interop store because node is stopping: " +
+                    store);
 
-        return createPlatformCache(cache);
+            if (started)
+                registerStore0(store, convertBinary);
+            else
+                pendingStores.add(new StoreInfo(store, convertBinary));
+        }
+        finally {
+            storeLock.readLock().unlock();
+        }
     }
 
     /** {@inheritDoc} */
-    @Override public PlatformTargetProxy createCache(@Nullable String name) throws IgniteCheckedException {
-        IgniteCacheProxy cache = (IgniteCacheProxy)ctx.grid().createCache(name);
-
-        assert cache != null;
+    @Override public void onDisconnected(IgniteFuture<?> reconnectFut) throws IgniteCheckedException {
+        platformCtx.gateway().onClientDisconnected();
 
-        return createPlatformCache(cache);
+        // 1) onReconnected is called on all grid components.
+        // 2) After all of grid components have completed their reconnection, reconnectFut is completed.
+        reconnectFut.listen(new CI1<IgniteFuture<?>>() {
+            @Override public void apply(IgniteFuture<?> future) {
+                platformCtx.gateway().onClientReconnected(clusterRestarted);
+            }
+        });
     }
 
     /** {@inheritDoc} */
-    @Override public PlatformTargetProxy getOrCreateCache(@Nullable String name) throws IgniteCheckedException {
-        IgniteCacheProxy cache = (IgniteCacheProxy)ctx.grid().getOrCreateCache(name);
+    @Override public IgniteInternalFuture<?> onReconnected(boolean clusterRestarted) throws IgniteCheckedException {
+        // Save the flag value for callback of reconnectFut.
+        this.clusterRestarted = clusterRestarted;
+
+        return null;
+    }
 
+    /**
+     * Creates new platform cache.
+     */
+    private PlatformTarget createPlatformCache(IgniteCacheProxy cache) {
         assert cache != null;
 
-        return createPlatformCache(cache);
+        return new PlatformCache(platformCtx, cache, false, cacheExts);
     }
 
-    /** {@inheritDoc} */
-    @Override public PlatformTargetProxy createCacheFromConfig(long memPtr) throws IgniteCheckedException {
-        BinaryRawReaderEx reader = platformCtx.reader(platformCtx.memory().get(memPtr));
-        CacheConfiguration cfg = PlatformConfigurationUtils.readCacheConfiguration(reader);
+    /**
+     * Checks whether logger level is enabled.
+     *
+     * @param level Level.
+     * @return Result.
+     */
+    private boolean loggerIsLevelEnabled(int level) {
+        IgniteLogger log = ctx.grid().log();
 
-        IgniteCacheProxy cache = reader.readBoolean()
-            ? (IgniteCacheProxy)ctx.grid().createCache(cfg, PlatformConfigurationUtils.readNearConfiguration(reader))
-            : (IgniteCacheProxy)ctx.grid().createCache(cfg);
+        switch (level) {
+            case PlatformLogger.LVL_TRACE:
+                return log.isTraceEnabled();
+            case PlatformLogger.LVL_DEBUG:
+                return log.isDebugEnabled();
+            case PlatformLogger.LVL_INFO:
+                return log.isInfoEnabled();
+            case PlatformLogger.LVL_WARN:
+                return true;
+            case PlatformLogger.LVL_ERROR:
+                return true;
+            default:
+                assert false;
+        }
 
-        return createPlatformCache(cache);
+        return false;
     }
 
-    /** {@inheritDoc} */
-    @Override public PlatformTargetProxy getOrCreateCacheFromConfig(long memPtr) throws IgniteCheckedException {
-        BinaryRawReaderEx reader = platformCtx.reader(platformCtx.memory().get(memPtr));
-        CacheConfiguration cfg = PlatformConfigurationUtils.readCacheConfiguration(reader);
+    /**
+     * Logs to the Ignite logger.
+     *
+     * @param level Level.
+     * @param message Message.
+     * @param category Category.
+     * @param errorInfo Exception.
+     */
+    private void loggerLog(int level, String message, String category, String errorInfo) {
+        IgniteLogger log = ctx.grid().log();
 
-        IgniteCacheProxy cache = reader.readBoolean()
-            ? (IgniteCacheProxy)ctx.grid().getOrCreateCache(cfg,
-                    PlatformConfigurationUtils.readNearConfiguration(reader))
-            : (IgniteCacheProxy)ctx.grid().getOrCreateCache(cfg);
+        if (category != null)
+            log = log.getLogger(category);
 
-        return createPlatformCache(cache);
-    }
+        Throwable err = errorInfo == null ? null : new IgniteException("Platform error:" + errorInfo);
 
-    /** {@inheritDoc} */
-    @Override public void destroyCache(@Nullable String name) throws IgniteCheckedException {
-        ctx.grid().destroyCache(name);
+        switch (level) {
+            case PlatformLogger.LVL_TRACE:
+                log.trace(message);
+                break;
+            case PlatformLogger.LVL_DEBUG:
+                log.debug(message);
+                break;
+            case PlatformLogger.LVL_INFO:
+                log.info(message);
+                break;
+            case PlatformLogger.LVL_WARN:
+                log.warning(message, err);
+                break;
+            case PlatformLogger.LVL_ERROR:
+                log.error(message, err);
+                break;
+            default:
+                assert false;
+        }
     }
 
     /** {@inheritDoc} */
-    @Override public PlatformTargetProxy affinity(@Nullable String name) throws IgniteCheckedException {
-        return proxy(new PlatformAffinity(platformCtx, ctx, name));
+    @Override public long processInLongOutLong(int type, long val) throws IgniteCheckedException {
+        switch (type) {
+            case OP_LOGGER_IS_LEVEL_ENABLED: {
+                return loggerIsLevelEnabled((int) val) ? PlatformAbstractTarget.TRUE : PlatformAbstractTarget.FALSE;
+            }
+
+            case OP_RELEASE_START: {
+                releaseStart();
+
+                return 0;
+            }
+        }
+
+        return PlatformAbstractTarget.throwUnsupported(type);
     }
 
     /** {@inheritDoc} */
-    @Override public PlatformTargetProxy dataStreamer(@Nullable String cacheName, boolean keepBinary)
-        throws IgniteCheckedException {
-        IgniteDataStreamer ldr = ctx.dataStream().dataStreamer(cacheName);
+    @Override public long processInStreamOutLong(int type, BinaryRawReaderEx reader) throws IgniteCheckedException {
+        switch (type) {
+            case OP_DESTROY_CACHE: {
+                ctx.grid().destroyCache(reader.readString());
+
+                return 0;
+            }
 
-        ldr.keepBinary(true);
+            case OP_LOGGER_LOG: {
+                loggerLog(reader.readInt(), reader.readString(), reader.readString(), reader.readString());
 
-        return proxy(new PlatformDataStreamer(platformCtx, cacheName, (DataStreamerImpl)ldr, keepBinary));
+                return 0;
+            }
+        }
+
+        return PlatformAbstractTarget.throwUnsupported(type);
     }
 
     /** {@inheritDoc} */
-    @Override public PlatformTargetProxy transactions() {
-        return proxy(new PlatformTransactions(platformCtx));
+    @Override public long processInStreamOutLong(int type, BinaryRawReaderEx reader, PlatformMemory mem) throws IgniteCheckedException {
+        return processInStreamOutLong(type, reader);
     }
 
     /** {@inheritDoc} */
-    @Override public PlatformTargetProxy projection() throws IgniteCheckedException {
-        return proxy(new PlatformClusterGroup(platformCtx, ctx.grid().cluster()));
+    @Override public void processInStreamOutStream(int type, BinaryRawReaderEx reader, BinaryRawWriterEx writer) throws IgniteCheckedException {
+        PlatformAbstractTarget.throwUnsupported(type);
     }
 
     /** {@inheritDoc} */
-    @Override public PlatformTargetProxy compute(PlatformTargetProxy grp) {
-        PlatformClusterGroup grp0 = (PlatformClusterGroup)grp.unwrap();
+    @Override public PlatformTarget processInStreamOutObject(int type, BinaryRawReaderEx reader) throws IgniteCheckedException {
+        switch (type) {
+            case OP_GET_CACHE: {
+                String name = reader.readString();
 
-        return proxy(new PlatformCompute(platformCtx, grp0.projection(), PlatformUtils.ATTR_PLATFORM));
-    }
+                IgniteCacheProxy cache = (IgniteCacheProxy)ctx.grid().cache(name);
 
-    /** {@inheritDoc} */
-    @Override public PlatformTargetProxy message(PlatformTargetProxy grp) {
-        PlatformClusterGroup grp0 = (PlatformClusterGroup)grp.unwrap();
+                if (cache == null)
+                    throw new IllegalArgumentException("Cache doesn't exist: " + name);
 
-        return proxy(new PlatformMessaging(platformCtx, grp0.projection().ignite().message(grp0.projection())));
-    }
+                return createPlatformCache(cache);
+            }
 
-    /** {@inheritDoc} */
-    @Override public PlatformTargetProxy events(PlatformTargetProxy grp) {
-        PlatformClusterGroup grp0 = (PlatformClusterGroup)grp.unwrap();
+            case OP_CREATE_CACHE: {
+                String name = reader.readString();
 
-        return proxy(new PlatformEvents(platformCtx, grp0.projection().ignite().events(grp0.projection())));
-    }
+                IgniteCacheProxy cache = (IgniteCacheProxy)ctx.grid().createCache(name);
 
-    /** {@inheritDoc} */
-    @Override public PlatformTargetProxy services(PlatformTargetProxy grp) {
-        PlatformClusterGroup grp0 = (PlatformClusterGroup)grp.unwrap();
+                return createPlatformCache(cache);
+            }
 
-        return proxy(new PlatformServices(platformCtx, grp0.projection().ignite().services(grp0.projection()), false));
-    }
+            case OP_GET_OR_CREATE_CACHE: {
+                String name = reader.readString();
 
-    /** {@inheritDoc} */
-    @Override public PlatformTargetProxy extensions() {
-        return null;
-    }
+                IgniteCacheProxy cache = (IgniteCacheProxy)ctx.grid().getOrCreateCache(name);
 
-    /** {@inheritDoc} */
-    @Override public PlatformTargetProxy extension(int id) {
-        if (extensions != null && id < extensions.length) {
-            PlatformPluginExtension ext = extensions[id];
+                return createPlatformCache(cache);
+            }
 
-            if (ext != null)
-                return proxy(ext.createTarget());
-        }
+            case OP_CREATE_CACHE_FROM_CONFIG: {
+                CacheConfiguration cfg = PlatformConfigurationUtils.readCacheConfiguration(reader);
 
-        throw new IgniteException("Platform extension is not registered [id=" + id + ']');
-    }
+                IgniteCacheProxy cache = reader.readBoolean()
+                        ? (IgniteCacheProxy)ctx.grid().createCache(cfg, PlatformConfigurationUtils.readNearConfiguration(reader))
+                        : (IgniteCacheProxy)ctx.grid().createCache(cfg);
 
-    /** {@inheritDoc} */
-    @Override public void registerStore(PlatformCacheStore store, boolean convertBinary)
-        throws IgniteCheckedException {
-        storeLock.readLock().lock();
+                return createPlatformCache(cache);
+            }
 
-        try {
-            if (stopped)
-                throw new IgniteCheckedException("Failed to initialize interop store because node is stopping: " +
-                    store);
+            case OP_GET_OR_CREATE_CACHE_FROM_CONFIG: {
+                CacheConfiguration cfg = PlatformConfigurationUtils.readCacheConfiguration(reader);
 
-            if (started)
-                registerStore0(store, convertBinary);
-            else
-                pendingStores.add(new StoreInfo(store, convertBinary));
-        }
-        finally {
-            storeLock.readLock().unlock();
-        }
-    }
+                IgniteCacheProxy cache = reader.readBoolean()
+                        ? (IgniteCacheProxy)ctx.grid().getOrCreateCache(cfg,
+                        PlatformConfigurationUtils.readNearConfiguration(reader))
+                        : (IgniteCacheProxy)ctx.grid().getOrCreateCache(cfg);
 
-    /** {@inheritDoc} */
-    @Override public PlatformTargetProxy atomicLong(String name, long initVal, boolean create) throws IgniteException {
-        GridCacheAtomicLongImpl atomicLong = (GridCacheAtomicLongImpl)ignite().atomicLong(name, initVal, create);
+                return createPlatformCache(cache);
+            }
 
-        if (atomicLong == null)
-            return null;
+            case OP_GET_AFFINITY: {
+                return new PlatformAffinity(platformCtx, ctx, reader.readString());
+            }
 
-        return proxy(new PlatformAtomicLong(platformCtx, atomicLong));
-    }
+            case OP_GET_DATA_STREAMER: {
+                String cacheName = reader.readString();
+                boolean keepBinary = reader.readBoolean();
 
-    /** {@inheritDoc} */
-    @Override public PlatformTargetProxy atomicSequence(String name, long initVal, boolean create)
-        throws IgniteException {
-        IgniteAtomicSequence atomicSeq = ignite().atomicSequence(name, initVal, create);
+                IgniteDataStreamer ldr = ctx.dataStream().dataStreamer(cacheName);
 
-        if (atomicSeq == null)
-            return null;
+                ldr.keepBinary(true);
 
-        return proxy(new PlatformAtomicSequence(platformCtx, atomicSeq));
-    }
+                return new PlatformDataStreamer(platformCtx, cacheName, (DataStreamerImpl)ldr, keepBinary);
+            }
 
-    /** {@inheritDoc} */
-    @Override public PlatformTargetProxy atomicReference(String name, long memPtr, boolean create)
-        throws IgniteException {
-        PlatformAtomicReference ref = PlatformAtomicReference.createInstance(platformCtx, name, memPtr, create);
+            case OP_GET_EXTENSION: {
+                int id = reader.readInt();
 
-        return ref != null ? proxy(ref) : null;
-    }
+                if (extensions != null && id < extensions.length) {
+                    PlatformPluginExtension ext = extensions[id];
 
-    /** {@inheritDoc} */
-    @Override public void onDisconnected(IgniteFuture<?> reconnectFut) throws IgniteCheckedException {
-        platformCtx.gateway().onClientDisconnected();
+                    if (ext != null) {
+                        return ext.createTarget();
+                    }
+                }
 
-        // 1) onReconnected is called on all grid components.
-        // 2) After all of grid components have completed their reconnection, reconnectFut is completed.
-        reconnectFut.listen(new CI1<IgniteFuture<?>>() {
-            @Override public void apply(IgniteFuture<?> future) {
-                platformCtx.gateway().onClientReconnected(clusterRestarted);
+                throw new IgniteException("Platform extension is not registered [id=" + id + ']');
             }
-        });
-    }
 
-    /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<?> onReconnected(boolean clusterRestarted) throws IgniteCheckedException {
-        // Save the flag value for callback of reconnectFut.
-        this.clusterRestarted = clusterRestarted;
+            case OP_GET_ATOMIC_LONG: {
+                String name = reader.readString();
+                long initVal = reader.readLong();
+                boolean create = reader.readBoolean();
 
-        return null;
-    }
+                GridCacheAtomicLongImpl atomicLong = (GridCacheAtomicLongImpl)ignite().atomicLong(name, initVal, create);
 
-    /** {@inheritDoc} */
-    @Override public void getIgniteConfiguration(long memPtr) {
-        PlatformOutputStream stream = platformCtx.memory().get(memPtr).output();
-        BinaryRawWriterEx writer = platformCtx.writer(stream);
+                if (atomicLong == null)
+                    return null;
 
-        PlatformConfigurationUtils.writeIgniteConfiguration(writer, ignite().configuration());
+                return new PlatformAtomicLong(platformCtx, atomicLong);
+            }
 
-        stream.synchronize();
-    }
+            case OP_GET_ATOMIC_REFERENCE: {
+                String name = reader.readString();
+                Object initVal = reader.readObjectDetached();
+                boolean create = reader.readBoolean();
 
-    /** {@inheritDoc} */
-    @Override public void getCacheNames(long memPtr) {
-        PlatformOutputStream stream = platformCtx.memory().get(memPtr).output();
-        BinaryRawWriterEx writer = platformCtx.writer(stream);
+                return PlatformAtomicReference.createInstance(platformCtx, name, initVal, create);
+            }
 
-        Collection<String> names = ignite().cacheNames();
+            case OP_GET_ATOMIC_SEQUENCE: {
+                String name = reader.readString();
+                long initVal = reader.readLong();
+                boolean create = reader.readBoolean();
 
-        writer.writeInt(names.size());
+                IgniteAtomicSequence atomicSeq = ignite().atomicSequence(name, initVal, create);
 
-        for (String name : names)
-            writer.writeString(name);
+                if (atomicSeq == null)
+                    return null;
 
-        stream.synchronize();
-    }
+                return new PlatformAtomicSequence(platformCtx, atomicSeq);
+            }
 
-    /** {@inheritDoc} */
-    @Override public PlatformTargetProxy createNearCache(@Nullable String cacheName, long memPtr) {
-        NearCacheConfiguration cfg = getNearCacheConfiguration(memPtr);
+            case OP_CREATE_NEAR_CACHE: {
+                String cacheName = reader.readString();
 
-        IgniteCacheProxy cache = (IgniteCacheProxy)ctx.grid().createNearCache(cacheName, cfg);
+                NearCacheConfiguration cfg = PlatformConfigurationUtils.readNearConfiguration(reader);
 
-        return createPlatformCache(cache);
-    }
+                IgniteCacheProxy cache = (IgniteCacheProxy)ctx.grid().createNearCache(cacheName, cfg);
 
-    /** {@inheritDoc} */
-    @Override public PlatformTargetProxy getOrCreateNearCache(@Nullable String cacheName, long memPtr) {
-        NearCacheConfiguration cfg = getNearCacheConfiguration(memPtr);
+                return createPlatformCache(cache);
+            }
+
+            case OP_GET_OR_CREATE_NEAR_CACHE: {
+                String cacheName = reader.readString();
 
-        IgniteCacheProxy cache = (IgniteCacheProxy)ctx.grid().getOrCreateNearCache(cacheName, cfg);
+                NearCacheConfiguration cfg = PlatformConfigurationUtils.readNearConfiguration(reader);
 
-        return createPlatformCache(cache);
+                IgniteCacheProxy cache = (IgniteCacheProxy)ctx.grid().getOrCreateNearCache(cacheName, cfg);
+
+                return createPlatformCache(cache);
+            }
+        }
+
+        return PlatformAbstractTarget.throwUnsupported(type);
     }
 
-    /**
-     * Creates new platform cache.
-     */
-    private PlatformTargetProxy createPlatformCache(IgniteCacheProxy cache) {
-        return proxy(new PlatformCache(platformCtx, cache, false, cacheExts));
+    /** {@inheritDoc} */
+    @Override public PlatformTarget processInObjectStreamOutObjectStream(int type, @Nullable PlatformTarget arg,
+                                                                         BinaryRawReaderEx reader,
+                                                                         BinaryRawWriterEx writer)
+            throws IgniteCheckedException {
+        return PlatformAbstractTarget.throwUnsupported(type);
     }
 
     /** {@inheritDoc} */
-    @Override public boolean loggerIsLevelEnabled(int level) {
-        IgniteLogger log = ctx.grid().log();
+    @Override public void processOutStream(int type, BinaryRawWriterEx writer) throws IgniteCheckedException {
+        switch (type) {
+            case OP_GET_IGNITE_CONFIGURATION: {
+                PlatformConfigurationUtils.writeIgniteConfiguration(writer, ignite().configuration());
 
-        switch (level) {
-            case PlatformLogger.LVL_TRACE:
-                return log.isTraceEnabled();
-            case PlatformLogger.LVL_DEBUG:
-                return log.isDebugEnabled();
-            case PlatformLogger.LVL_INFO:
-                return log.isInfoEnabled();
-            case PlatformLogger.LVL_WARN:
-                return true;
-            case PlatformLogger.LVL_ERROR:
-                return true;
-            default:
-                assert false;
+                return;
+            }
+
+            case OP_GET_CACHE_NAMES: {
+                Collection<String> names = ignite().cacheNames();
+
+                writer.writeInt(names.size());
+
+                for (String name : names)
+                    writer.writeString(name);
+
+                return;
+            }
         }
 
-        return false;
+        PlatformAbstractTarget.throwUnsupported(type);
     }
 
     /** {@inheritDoc} */
-    @Override public void loggerLog(int level, String message, String category, String errorInfo) {
-        IgniteLogger log = ctx.grid().log();
+    @Override public PlatformTarget processOutObject(int type) throws IgniteCheckedException {
+        switch (type) {
+            case OP_GET_TRANSACTIONS:
+                return new PlatformTransactions(platformCtx);
 
-        if (category != null)
-            log = log.getLogger(category);
+            case OP_GET_CLUSTER_GROUP:
+                return new PlatformClusterGroup(platformCtx, ctx.grid().cluster());
 
-        Throwable err = errorInfo == null ? null : new IgniteException("Platform error:" + errorInfo);
-
-        switch (level) {
-            case PlatformLogger.LVL_TRACE:
-                log.trace(message);
-                break;
-            case PlatformLogger.LVL_DEBUG:
-                log.debug(message);
-                break;
-            case PlatformLogger.LVL_INFO:
-                log.info(message);
-                break;
-            case PlatformLogger.LVL_WARN:
-                log.warning(message, err);
-                break;
-            case PlatformLogger.LVL_ERROR:
-                log.error(message, err);
-                break;
-            default:
-                assert false;
+            case OP_GET_BINARY_PROCESSOR: {
+                return new PlatformBinaryProcessor(platformCtx);
+            }
         }
+
+        return PlatformAbstractTarget.throwUnsupported(type);
     }
 
     /** {@inheritDoc} */
-    @Override public PlatformTargetProxy binaryProcessor() {
-        return proxy(new PlatformBinaryProcessor(platformCtx));
+    @Override public PlatformAsyncResult processInStreamAsync(int type, BinaryRawReaderEx reader) throws IgniteCheckedException {
+        return PlatformAbstractTarget.throwUnsupported(type);
     }
 
-    /**
-     * Gets the near cache config.
-     *
-     * @param memPtr Memory pointer.
-     * @return Near config.
-     */
-    private NearCacheConfiguration getNearCacheConfiguration(long memPtr) {
-        assert memPtr != 0;
-
-        BinaryRawReaderEx reader = platformCtx.reader(platformCtx.memory().get(memPtr));
-        return PlatformConfigurationUtils.readNearConfiguration(reader);
+    /** {@inheritDoc} */
+    @Override public Exception convertException(Exception e) {
+        return e;
     }
 
     /**
@@ -635,13 +736,6 @@ public class PlatformProcessorImpl extends GridProcessorAdapter implements Platf
     }
 
     /**
-     * Wraps target in a proxy.
-     */
-    private PlatformTargetProxy proxy(PlatformTarget target) {
-        return new PlatformTargetProxyImpl(target, platformCtx);
-    }
-
-    /**
      * Store and manager pair.
      */
     private static class StoreInfo {

http://git-wip-us.apache.org/repos/asf/ignite/blob/bdc9e4b9/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java
index f6e3d2e..7c1c03e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java
@@ -36,6 +36,10 @@ import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget;
 import org.apache.ignite.internal.processors.platform.PlatformContext;
 import org.apache.ignite.internal.processors.platform.PlatformTarget;
 import org.apache.ignite.internal.processors.platform.cache.PlatformCache;
+import org.apache.ignite.internal.processors.platform.compute.PlatformCompute;
+import org.apache.ignite.internal.processors.platform.events.PlatformEvents;
+import org.apache.ignite.internal.processors.platform.messaging.PlatformMessaging;
+import org.apache.ignite.internal.processors.platform.services.PlatformServices;
 import org.apache.ignite.internal.processors.platform.utils.PlatformUtils;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.jetbrains.annotations.Nullable;
@@ -126,6 +130,19 @@ public class PlatformClusterGroup extends PlatformAbstractTarget {
     /** */
     private static final int OP_PERSISTENT_STORE_METRICS = 30;
 
+    /** */
+    private static final int OP_GET_COMPUTE = 31;
+
+    /** */
+    private static final int OP_GET_MESSAGING = 32;
+
+    /** */
+    private static final int OP_GET_EVENTS = 33;
+
+    /** */
+    private static final int OP_GET_SERVICES = 34;
+
+
     /** Projection. */
     private final ClusterGroupEx prj;
 
@@ -381,6 +398,18 @@ public class PlatformClusterGroup extends PlatformAbstractTarget {
 
             case OP_FOR_SERVERS:
                 return new PlatformClusterGroup(platformCtx, (ClusterGroupEx)prj.forServers());
+
+            case OP_GET_COMPUTE:
+                return new PlatformCompute(platformCtx, prj, PlatformUtils.ATTR_PLATFORM);
+
+            case OP_GET_MESSAGING:
+                return new PlatformMessaging(platformCtx, platformCtx.kernalContext().grid().message(prj));
+
+            case OP_GET_EVENTS:
+                return new PlatformEvents(platformCtx, platformCtx.kernalContext().grid().events(prj));
+
+            case OP_GET_SERVICES:
+                return new PlatformServices(platformCtx, platformCtx.kernalContext().grid().services(prj),false);
         }
 
         return super.processOutObject(type);

http://git-wip-us.apache.org/repos/asf/ignite/blob/bdc9e4b9/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicReference.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicReference.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicReference.java
index a644259..93c0040 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicReference.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicReference.java
@@ -23,7 +23,6 @@ import org.apache.ignite.internal.binary.BinaryRawWriterEx;
 import org.apache.ignite.internal.processors.datastructures.GridCacheAtomicReferenceImpl;
 import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget;
 import org.apache.ignite.internal.processors.platform.PlatformContext;
-import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
 
 /**
  * Platform atomic reference wrapper.
@@ -53,23 +52,15 @@ public class PlatformAtomicReference extends PlatformAbstractTarget {
      *
      * @param ctx Context.
      * @param name Name.
-     * @param memPtr Pointer to a stream with initial value. 0 for default value.
+     * @param initVal Initial value.
      * @param create Create flag.
      * @return Instance of a PlatformAtomicReference, or null when Ignite reference with specific name is null.
      */
-    public static PlatformAtomicReference createInstance(PlatformContext ctx, String name, long memPtr,
+    public static PlatformAtomicReference createInstance(PlatformContext ctx, String name, Object initVal,
         boolean create) {
         assert ctx != null;
         assert name != null;
 
-        Object initVal = null;
-
-        if (memPtr != 0) {
-            try (PlatformMemory mem = ctx.memory().get(memPtr)) {
-                initVal = ctx.reader(mem).readObjectDetached();
-            }
-        }
-
         GridCacheAtomicReferenceImpl atomicRef =
             (GridCacheAtomicReferenceImpl)ctx.kernalContext().grid().atomicReference(name, initVal, create);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/bdc9e4b9/modules/platforms/cpp/core/include/ignite/ignite.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/ignite.h b/modules/platforms/cpp/core/include/ignite/ignite.h
index b3b06f0..07134a1 100644
--- a/modules/platforms/cpp/core/include/ignite/ignite.h
+++ b/modules/platforms/cpp/core/include/ignite/ignite.h
@@ -102,7 +102,7 @@ namespace ignite
         template<typename K, typename V>
         cache::Cache<K, V> GetCache(const char* name, IgniteError& err)
         {
-            impl::cache::CacheImpl* cacheImpl = impl.Get()->GetCache<K, V>(name, err);
+            impl::cache::CacheImpl* cacheImpl = impl.Get()->GetCache(name, err);
 
             return cache::Cache<K, V>(cacheImpl);
         }
@@ -139,7 +139,7 @@ namespace ignite
         template<typename K, typename V>
         cache::Cache<K, V> GetOrCreateCache(const char* name, IgniteError& err)
         {
-            impl::cache::CacheImpl* cacheImpl = impl.Get()->GetOrCreateCache<K, V>(name, err);
+            impl::cache::CacheImpl* cacheImpl = impl.Get()->GetOrCreateCache(name, err);
 
             return cache::Cache<K, V>(cacheImpl);
         }
@@ -176,7 +176,7 @@ namespace ignite
         template<typename K, typename V>
         cache::Cache<K, V> CreateCache(const char* name, IgniteError& err)
         {
-            impl::cache::CacheImpl* cacheImpl = impl.Get()->CreateCache<K, V>(name, err);
+            impl::cache::CacheImpl* cacheImpl = impl.Get()->CreateCache(name, err);
 
             return cache::Cache<K, V>(cacheImpl);
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/bdc9e4b9/modules/platforms/cpp/core/include/ignite/impl/compute/compute_impl.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/impl/compute/compute_impl.h b/modules/platforms/cpp/core/include/ignite/impl/compute/compute_impl.h
index 4ba1c1c..2b04dcb 100644
--- a/modules/platforms/cpp/core/include/ignite/impl/compute/compute_impl.h
+++ b/modules/platforms/cpp/core/include/ignite/impl/compute/compute_impl.h
@@ -208,7 +208,10 @@ namespace ignite
 
                     out.Synchronize();
 
-                    jobject target = InStreamOutObject(operation, *mem.Get());
+                    IgniteError err;
+                    jobject target = InStreamOutObject(operation, *mem.Get(), err);
+                    IgniteError::ThrowIfNeeded(err);
+
                     std::auto_ptr<common::Cancelable> cancelable(new CancelableImpl(GetEnvironmentPointer(), target));
 
                     return cancelable;

http://git-wip-us.apache.org/repos/asf/ignite/blob/bdc9e4b9/modules/platforms/cpp/core/include/ignite/impl/ignite_impl.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/impl/ignite_impl.h b/modules/platforms/cpp/core/include/ignite/impl/ignite_impl.h
index baddec4..d1763c4 100644
--- a/modules/platforms/cpp/core/include/ignite/impl/ignite_impl.h
+++ b/modules/platforms/cpp/core/include/ignite/impl/ignite_impl.h
@@ -28,14 +28,34 @@
 #include <ignite/impl/cluster/cluster_group_impl.h>
 #include <ignite/impl/compute/compute_impl.h>
 
+using namespace ignite::impl::interop;
+using namespace ignite::common::concurrent;
+using namespace ignite::impl::binary;
+using namespace ignite::binary;
+
 namespace ignite 
 {
     namespace impl 
     {
+        /*
+        * PlatformProcessor op codes.
+        */
+        struct ProcessorOp
+        {
+            enum Type
+            {
+                GET_CACHE = 1,
+                CREATE_CACHE = 2,
+                GET_OR_CREATE_CACHE = 3,
+                GET_TRANSACTIONS = 9,
+                GET_CLUSTER_GROUP = 10,
+            };
+        };
+
         /**
          * Ignite implementation.
          */
-        class IGNITE_FRIEND_EXPORT IgniteImpl
+        class IGNITE_FRIEND_EXPORT IgniteImpl : private interop::InteropTarget
         {
             typedef common::concurrent::SharedPointer<IgniteEnvironment> SP_IgniteEnvironment;
             typedef common::concurrent::SharedPointer<transactions::TransactionsImpl> SP_TransactionsImpl;
@@ -48,14 +68,9 @@ namespace ignite
              * @param env Environment.
              * @param javaRef Reference to java object.
              */
-            IgniteImpl(SP_IgniteEnvironment env, jobject javaRef);
+            IgniteImpl(SP_IgniteEnvironment env);
             
             /**
-             * Destructor.
-             */
-            ~IgniteImpl();
-
-            /**
              * Get name of the Ignite.
              *
              * @return Name.
@@ -82,23 +97,9 @@ namespace ignite
              * @param name Cache name.
              * @param err Error.
              */
-            template<typename K, typename V> 
             cache::CacheImpl* GetCache(const char* name, IgniteError& err)
             {
-                ignite::jni::java::JniErrorInfo jniErr;
-
-                jobject cacheJavaRef = env.Get()->Context()->ProcessorCache(javaRef, name, &jniErr);
-
-                if (!cacheJavaRef)
-                {
-                    IgniteError::SetError(jniErr.code, jniErr.errCls, jniErr.errMsg, err);
-
-                    return NULL;
-                }
-
-                char* name0 = common::CopyChars(name);
-
-                return new cache::CacheImpl(name0, env, cacheJavaRef);
+                return GetOrCreateCache(name, err, ProcessorOp::GET_CACHE);
             }
 
             /**
@@ -107,23 +108,9 @@ namespace ignite
              * @param name Cache name.
              * @param err Error.
              */
-            template<typename K, typename V>
             cache::CacheImpl* GetOrCreateCache(const char* name, IgniteError& err)
             {
-                ignite::jni::java::JniErrorInfo jniErr;
-
-                jobject cacheJavaRef = env.Get()->Context()->ProcessorGetOrCreateCache(javaRef, name, &jniErr);
-
-                if (!cacheJavaRef)
-                {
-                    IgniteError::SetError(jniErr.code, jniErr.errCls, jniErr.errMsg, err);
-
-                    return NULL;
-                }
-
-                char* name0 = common::CopyChars(name);
-
-                return new cache::CacheImpl(name0, env, cacheJavaRef);
+                return GetOrCreateCache(name, err, ProcessorOp::GET_OR_CREATE_CACHE);
             }
 
             /**
@@ -132,23 +119,9 @@ namespace ignite
              * @param name Cache name.
              * @param err Error.
              */
-            template<typename K, typename V>
             cache::CacheImpl* CreateCache(const char* name, IgniteError& err)
             {
-                ignite::jni::java::JniErrorInfo jniErr;
-
-                jobject cacheJavaRef = env.Get()->Context()->ProcessorCreateCache(javaRef, name, &jniErr);
-
-                if (!cacheJavaRef)
-                {
-                    IgniteError::SetError(jniErr.code, jniErr.errCls, jniErr.errMsg, err);
-
-                    return NULL;
-                }
-
-                char* name0 = common::CopyChars(name);
-
-                return new cache::CacheImpl(name0, env, cacheJavaRef);
+                return GetOrCreateCache(name, err, ProcessorOp::CREATE_CACHE);
             }
 
             /**
@@ -227,9 +200,6 @@ namespace ignite
             /** Environment. */
             SP_IgniteEnvironment env;
 
-            /** Native Java counterpart. */
-            jobject javaRef;
-
             /** Transactions implementaion. */
             SP_TransactionsImpl txImpl;
 
@@ -237,6 +207,38 @@ namespace ignite
             cluster::SP_ClusterGroupImpl prjImpl;
 
             IGNITE_NO_COPY_ASSIGNMENT(IgniteImpl)
+
+            /**
+            * Get or create cache.
+            *
+            * @param name Cache name.
+            * @param err Error.
+            * @param op Operation code.
+            */
+            cache::CacheImpl* GetOrCreateCache(const char* name, IgniteError& err, int32_t op)
+            {
+                SharedPointer<InteropMemory> mem = env.Get()->AllocateMemory();
+                InteropMemory* mem0 = mem.Get();
+                InteropOutputStream out(mem0);
+                BinaryWriterImpl writer(&out, env.Get()->GetTypeManager());
+                BinaryRawWriter rawWriter(&writer);
+
+                rawWriter.WriteString(name);
+
+                out.Synchronize();
+
+                jobject cacheJavaRef = InStreamOutObject(op, *mem0, err);
+
+                if (!cacheJavaRef)
+                {
+                    return NULL;
+                }
+
+                char* name0 = common::CopyChars(name);
+
+                return new cache::CacheImpl(name0, env, cacheJavaRef);
+            }
+
         };
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/bdc9e4b9/modules/platforms/cpp/core/include/ignite/impl/interop/interop_target.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/impl/interop/interop_target.h b/modules/platforms/cpp/core/include/ignite/impl/interop/interop_target.h
index 0384dcc..29a3a2d 100644
--- a/modules/platforms/cpp/core/include/ignite/impl/interop/interop_target.h
+++ b/modules/platforms/cpp/core/include/ignite/impl/interop/interop_target.h
@@ -60,6 +60,16 @@ namespace ignite
                 InteropTarget(ignite::common::concurrent::SharedPointer<IgniteEnvironment> env, jobject javaRef);
 
                 /**
+                * Constructor used to create new instance.
+                *
+                * @param env Environment.
+                * @param javaRef Reference to java object.
+                * @param javaRef Whether javaRef release in destructor should be skipped.
+                */
+                InteropTarget(ignite::common::concurrent::SharedPointer<IgniteEnvironment> env, jobject javaRef, 
+                    bool skipJavaRefRelease);
+
+                /**
                  * Destructor.
                  */
                 virtual ~InteropTarget();
@@ -139,9 +149,10 @@ namespace ignite
                  *
                  * @param opType Type of operation.
                  * @param outInMem Input and output memory.
+                 * @param err Error.
                  * @return Java object references.
                  */
-                jobject InStreamOutObject(int32_t opType, InteropMemory& outInMem);
+                jobject InStreamOutObject(int32_t opType, InteropMemory& outInMem, IgniteError& err);
 
                 /**
                 * Internal out-in operation.
@@ -190,6 +201,9 @@ namespace ignite
                 /** Handle to Java object. */
                 jobject javaRef;
 
+                /** javaRef release flag. */
+                bool skipJavaRefRelease;
+
                 IGNITE_NO_COPY_ASSIGNMENT(InteropTarget)
 
                 /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/bdc9e4b9/modules/platforms/cpp/core/src/ignition.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/src/ignition.cpp b/modules/platforms/cpp/core/src/ignition.cpp
index e12a208..bc25b07 100644
--- a/modules/platforms/cpp/core/src/ignition.cpp
+++ b/modules/platforms/cpp/core/src/ignition.cpp
@@ -301,13 +301,13 @@ namespace ignite
         stream.WriteBool(false);
         stream.Synchronize();
 
-        jobject javaRef = ctx.Get()->IgnitionStart(&springCfgPath0[0], namep, 2, mem.PointerLong(), &jniErr);
+        ctx.Get()->IgnitionStart(&springCfgPath0[0], namep, 2, mem.PointerLong(), &jniErr);
 
         // Releasing control over environment as it is controlled by Java at this point.
         // Even if the call has failed environment are going to be released by the Java.
         envTarget.release();
 
-        if (!javaRef)
+        if (!env.Get()->GetProcessor())
         {
             IgniteError::SetError(jniErr.code, jniErr.errCls, jniErr.errMsg, err);
 
@@ -323,7 +323,7 @@ namespace ignite
 
         env.Get()->ProcessorReleaseStart();
 
-        IgniteImpl* impl = new IgniteImpl(env, javaRef);
+        IgniteImpl* impl = new IgniteImpl(env);
 
         return Ignite(impl);
     }
@@ -383,22 +383,9 @@ namespace ignite
                         SharedPointer<IgniteEnvironment>* env =
                             static_cast<SharedPointer<IgniteEnvironment>*>(hnds->target);
 
-                        // 4. Get fresh node reference.
-                        jobject ref = ctx.Get()->IgnitionInstance(name0, &jniErr);
+                        IgniteImpl* impl = new IgniteImpl(*env);
 
-                        if (err.GetCode() == IgniteError::IGNITE_SUCCESS) {
-                            if (ref)
-                            {
-                                IgniteImpl* impl = new IgniteImpl(*env, ref);
-
-                                res = Ignite(impl);
-                            }
-                            else
-                                // Error: concurrent node stop.
-                                err = IgniteError(IgniteError::IGNITE_ERR_GENERIC,
-                                    "Failed to get Ignite instance because it was stopped concurrently.");
-
-                        }
+                        res = Ignite(impl);
                     }
                     else
                         // Error: no node with the given name.

http://git-wip-us.apache.org/repos/asf/ignite/blob/bdc9e4b9/modules/platforms/cpp/core/src/impl/ignite_environment.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/src/impl/ignite_environment.cpp b/modules/platforms/cpp/core/src/impl/ignite_environment.cpp
index 4e78f09..e5ce004 100644
--- a/modules/platforms/cpp/core/src/impl/ignite_environment.cpp
+++ b/modules/platforms/cpp/core/src/impl/ignite_environment.cpp
@@ -61,7 +61,30 @@ namespace ignite
                 ON_START = 49,
                 ON_STOP = 50,
                 COMPUTE_TASK_LOCAL_JOB_RESULT = 60,
-                COMPUTE_JOB_EXECUTE_LOCAL = 61
+                COMPUTE_JOB_EXECUTE_LOCAL = 61,
+            };
+        };
+
+        /*
+        * PlatformProcessor op codes.
+        */
+        struct ProcessorOp
+        {
+            enum Type
+            {
+                GET_BINARY_PROCESSOR = 21,
+                RELEASE_START = 22
+            };
+        };
+
+        /*
+         * PlatformClusterGroup op codes.
+         */
+        struct ClusterGroupOp
+        {
+            enum Type
+            {
+                GET_COMPUTE = 31
             };
         };
 
@@ -298,7 +321,10 @@ namespace ignite
         {
             latch.CountDown();
 
-            jobject binaryProc = Context()->ProcessorBinaryProcessor(proc.Get());
+            JniErrorInfo jniErr;
+
+            jobject binaryProc = Context()->TargetOutObject(proc.Get(), ProcessorOp::GET_BINARY_PROCESSOR, &jniErr);
+
             metaUpdater = new BinaryTypeUpdaterImpl(*this, binaryProc);
 
             metaMgr->SetUpdater(metaUpdater);
@@ -375,7 +401,7 @@ namespace ignite
         {
             JniErrorInfo jniErr;
 
-            jobject res = ctx.Get()->ProcessorCompute(proc.Get(), proj, &jniErr);
+            jobject res = ctx.Get()->TargetOutObject(proj, ClusterGroupOp::GET_COMPUTE, &jniErr);
 
             IgniteError err;
 
@@ -557,7 +583,10 @@ namespace ignite
         void IgniteEnvironment::ProcessorReleaseStart()
         {
             if (proc.Get())
-                ctx.Get()->ProcessorReleaseStart(proc.Get());
+            {
+                JniErrorInfo jniErr;
+                ctx.Get()->TargetInLongOutLong(proc.Get(), ProcessorOp::RELEASE_START, 0, &jniErr);
+            }
         }
 
         HandleRegistry& IgniteEnvironment::GetHandleRegistry()

http://git-wip-us.apache.org/repos/asf/ignite/blob/bdc9e4b9/modules/platforms/cpp/core/src/impl/ignite_impl.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/src/impl/ignite_impl.cpp b/modules/platforms/cpp/core/src/impl/ignite_impl.cpp
index 16e954c..f7ff185 100644
--- a/modules/platforms/cpp/core/src/impl/ignite_impl.cpp
+++ b/modules/platforms/cpp/core/src/impl/ignite_impl.cpp
@@ -24,9 +24,9 @@ namespace ignite
 {    
     namespace impl
     {
-        IgniteImpl::IgniteImpl(SharedPointer<IgniteEnvironment> env, jobject javaRef) :
-            env(env),
-            javaRef(javaRef)
+        IgniteImpl::IgniteImpl(SharedPointer<IgniteEnvironment> env) :
+            InteropTarget(env, static_cast<jobject>(env.Get()->GetProcessor()), true),
+            env(env)
         {
             IgniteError err;
 
@@ -39,11 +39,6 @@ namespace ignite
             IgniteError::ThrowIfNeeded(err);
         }
 
-        IgniteImpl::~IgniteImpl()
-        {
-            JniContext::Release(javaRef);
-        }
-
         const char* IgniteImpl::GetName() const
         {
             return env.Get()->InstanceName();
@@ -75,14 +70,10 @@ namespace ignite
         {
             SP_TransactionsImpl res;
 
-            JniErrorInfo jniErr;
-
-            jobject txJavaRef = env.Get()->Context()->ProcessorTransactions(javaRef, &jniErr);
+            jobject txJavaRef = InOpObject(ProcessorOp::GET_TRANSACTIONS, err);
 
             if (txJavaRef)
                 res = SP_TransactionsImpl(new transactions::TransactionsImpl(env, txJavaRef));
-            else
-                IgniteError::SetError(jniErr.code, jniErr.errCls, jniErr.errMsg, err);
 
             return res;
         }
@@ -93,12 +84,10 @@ namespace ignite
 
             JniErrorInfo jniErr;
 
-            jobject txJavaRef = env.Get()->Context()->ProcessorProjection(javaRef, &jniErr);
+            jobject clusterGroupJavaRef = InOpObject(ProcessorOp::GET_CLUSTER_GROUP, err);
 
-            if (txJavaRef)
-                res = cluster::SP_ClusterGroupImpl(new cluster::ClusterGroupImpl(env, txJavaRef));
-            else
-                IgniteError::SetError(jniErr.code, jniErr.errCls, jniErr.errMsg, err);
+            if (clusterGroupJavaRef)
+                res = cluster::SP_ClusterGroupImpl(new cluster::ClusterGroupImpl(env, clusterGroupJavaRef));
 
             return res;
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/bdc9e4b9/modules/platforms/cpp/core/src/impl/interop/interop_target.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/src/impl/interop/interop_target.cpp b/modules/platforms/cpp/core/src/impl/interop/interop_target.cpp
index 7eed6f3..70db2c9 100644
--- a/modules/platforms/cpp/core/src/impl/interop/interop_target.cpp
+++ b/modules/platforms/cpp/core/src/impl/interop/interop_target.cpp
@@ -32,14 +32,24 @@ namespace ignite
         namespace interop
         {
             InteropTarget::InteropTarget(SharedPointer<IgniteEnvironment> env, jobject javaRef) :
-                env(env), javaRef(javaRef)
+                env(env), javaRef(javaRef), skipJavaRefRelease(false)
+            {
+                // No-op.
+            }
+
+            InteropTarget::InteropTarget(SharedPointer<IgniteEnvironment> env, jobject javaRef, 
+                bool skipJavaRefRelease) :
+                env(env), javaRef(javaRef), skipJavaRefRelease(skipJavaRefRelease)
             {
                 // No-op.
             }
 
             InteropTarget::~InteropTarget()
             {
-                JniContext::Release(javaRef);
+                if (!skipJavaRefRelease) 
+                {
+                    JniContext::Release(javaRef);
+                }
             }
 
             int64_t InteropTarget::WriteTo(InteropMemory* mem, InputOperation& inOp, IgniteError& err)
@@ -216,7 +226,7 @@ namespace ignite
                 return OperationResult::AI_ERROR;
             }
 
-            jobject InteropTarget::InStreamOutObject(int32_t opType, InteropMemory& outInMem)
+            jobject InteropTarget::InStreamOutObject(int32_t opType, InteropMemory& outInMem, IgniteError& err)
             {
                 JniErrorInfo jniErr;
 
@@ -226,9 +236,7 @@ namespace ignite
                 {
                     jobject res = env.Get()->Context()->TargetInStreamOutObject(javaRef, opType, outInPtr, &jniErr);
 
-                    IgniteError err;
                     IgniteError::SetError(jniErr.code, jniErr.errCls, jniErr.errMsg, err);
-                    IgniteError::ThrowIfNeeded(err);
 
                     return res;
                 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/bdc9e4b9/modules/platforms/cpp/jni/include/ignite/jni/exports.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/jni/include/ignite/jni/exports.h b/modules/platforms/cpp/jni/include/ignite/jni/exports.h
index 6fe91c7..ea0c32a 100644
--- a/modules/platforms/cpp/jni/include/ignite/jni/exports.h
+++ b/modules/platforms/cpp/jni/include/ignite/jni/exports.h
@@ -25,40 +25,11 @@ namespace gcj = ignite::jni::java;
 extern "C" {
     int IGNITE_CALL IgniteReallocate(long long memPtr, int cap);
 
-    void* IGNITE_CALL IgniteIgnitionStart(gcj::JniContext* ctx, char* cfgPath, char* name, int factoryId, long long dataPtr);
-    void* IGNITE_CALL IgniteIgnitionInstance(gcj::JniContext* ctx, char* name);
+    void IGNITE_CALL IgniteIgnitionStart(gcj::JniContext* ctx, char* cfgPath, char* name, int factoryId, long long dataPtr);
     long long IGNITE_CALL IgniteIgnitionEnvironmentPointer(gcj::JniContext* ctx, char* name);
     bool IGNITE_CALL IgniteIgnitionStop(gcj::JniContext* ctx, char* name, bool cancel);
     void IGNITE_CALL IgniteIgnitionStopAll(gcj::JniContext* ctx, bool cancel);
 
-    void IGNITE_CALL IgniteProcessorReleaseStart(gcj::JniContext* ctx, void* obj);
-    void* IGNITE_CALL IgniteProcessorProjection(gcj::JniContext* ctx, void* obj);
-    void* IGNITE_CALL IgniteProcessorCache(gcj::JniContext* ctx, void* obj, char* name);
-    void* IGNITE_CALL IgniteProcessorCreateCache(gcj::JniContext* ctx, void* obj, char* name);
-    void* IGNITE_CALL IgniteProcessorGetOrCreateCache(gcj::JniContext* ctx, void* obj, char* name);
-    void* IGNITE_CALL IgniteProcessorCreateCacheFromConfig(gcj::JniContext* ctx, void* obj, long long memPtr);
-    void* IGNITE_CALL IgniteProcessorGetOrCreateCacheFromConfig(gcj::JniContext* ctx, void* obj, long long memPtr);
-    void* IGNITE_CALL IgniteProcessorCreateNearCache(gcj::JniContext* ctx, void* obj, char* name, long long memPtr);
-    void* IGNITE_CALL IgniteProcessorGetOrCreateNearCache(gcj::JniContext* ctx, void* obj, char* name, long long memPtr);
-    void IGNITE_CALL IgniteProcessorDestroyCache(gcj::JniContext* ctx, void* obj, char* name);
-    void* IGNITE_CALL IgniteProcessorAffinity(gcj::JniContext* ctx, void* obj, char* name);
-    void* IGNITE_CALL IgniteProcessorDataStreamer(gcj::JniContext* ctx, void* obj, char* name, bool keepPortable);
-    void* IGNITE_CALL IgniteProcessorTransactions(gcj::JniContext* ctx, void* obj);
-    void* IGNITE_CALL IgniteProcessorCompute(gcj::JniContext* ctx, void* obj, void* prj);
-    void* IGNITE_CALL IgniteProcessorMessage(gcj::JniContext* ctx, void* obj, void* prj);
-    void* IGNITE_CALL IgniteProcessorEvents(gcj::JniContext* ctx, void* obj, void* prj);
-    void* IGNITE_CALL IgniteProcessorServices(gcj::JniContext* ctx, void* obj, void* prj);
-    void* IGNITE_CALL IgniteProcessorExtensions(gcj::JniContext* ctx, void* obj);
-    void* IGNITE_CALL IgniteProcessorExtension(gcj::JniContext* ctx, void* obj, int id);
-    void* IGNITE_CALL IgniteProcessorAtomicLong(gcj::JniContext* ctx, void* obj, char* name, long long initVal, bool create);
-    void* IGNITE_CALL IgniteProcessorAtomicSequence(gcj::JniContext* ctx, void* obj, char* name, long long initVal, bool create);
-    void* IGNITE_CALL IgniteProcessorAtomicReference(gcj::JniContext* ctx, void* obj, char* name, long long memPtr, bool create);
-    void IGNITE_CALL IgniteProcessorGetIgniteConfiguration(gcj::JniContext* ctx, void* obj, long long memPtr);
-    void IGNITE_CALL IgniteProcessorGetCacheNames(gcj::JniContext* ctx, void* obj, long long memPtr);
-    bool IGNITE_CALL IgniteProcessorLoggerIsLevelEnabled(gcj::JniContext* ctx, void* obj, int level);
-    void IGNITE_CALL IgniteProcessorLoggerLog(gcj::JniContext* ctx, void* obj, int level, char* message, char* category, char* errorInfo);
-    void* IGNITE_CALL IgniteProcessorBinaryProcessor(gcj::JniContext* ctx, void* obj);
-
     long long IGNITE_CALL IgniteTargetInLongOutLong(gcj::JniContext* ctx, void* obj, int opType, long long memPtr);
     long long IGNITE_CALL IgniteTargetInStreamOutLong(gcj::JniContext* ctx, void* obj, int opType, long long memPtr);
     void IGNITE_CALL IgniteTargetInStreamOutStream(gcj::JniContext* ctx, void* obj, int opType, long long inMemPtr, long long outMemPtr);

http://git-wip-us.apache.org/repos/asf/ignite/blob/bdc9e4b9/modules/platforms/cpp/jni/include/ignite/jni/java.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/jni/include/ignite/jni/java.h b/modules/platforms/cpp/jni/include/ignite/jni/java.h
index f6d7207..c170a5b 100644
--- a/modules/platforms/cpp/jni/include/ignite/jni/java.h
+++ b/modules/platforms/cpp/jni/include/ignite/jni/java.h
@@ -175,32 +175,6 @@ namespace ignite
 
                 jclass c_PlatformProcessor;
                 jmethodID m_PlatformProcessor_releaseStart;
-                jmethodID m_PlatformProcessor_cache;
-                jmethodID m_PlatformProcessor_createCache;
-                jmethodID m_PlatformProcessor_getOrCreateCache;
-                jmethodID m_PlatformProcessor_createCacheFromConfig;
-                jmethodID m_PlatformProcessor_getOrCreateCacheFromConfig;
-                jmethodID m_PlatformProcessor_createNearCache;
-                jmethodID m_PlatformProcessor_getOrCreateNearCache;
-                jmethodID m_PlatformProcessor_destroyCache;
-                jmethodID m_PlatformProcessor_affinity;
-                jmethodID m_PlatformProcessor_dataStreamer;
-                jmethodID m_PlatformProcessor_transactions;
-                jmethodID m_PlatformProcessor_projection;
-                jmethodID m_PlatformProcessor_compute;
-                jmethodID m_PlatformProcessor_message;
-                jmethodID m_PlatformProcessor_events;
-                jmethodID m_PlatformProcessor_services;
-                jmethodID m_PlatformProcessor_extensions;
-                jmethodID m_PlatformProcessor_extension;
-                jmethodID m_PlatformProcessor_atomicLong;
-                jmethodID m_PlatformProcessor_getIgniteConfiguration;
-                jmethodID m_PlatformProcessor_getCacheNames;
-                jmethodID m_PlatformProcessor_atomicSequence;
-                jmethodID m_PlatformProcessor_atomicReference;
-                jmethodID m_PlatformProcessor_loggerIsLevelEnabled;
-                jmethodID m_PlatformProcessor_loggerLog;
-                jmethodID m_PlatformProcessor_binaryProcessor;
 
                 jclass c_PlatformTarget;
                 jmethodID m_PlatformTarget_inLongOutLong;
@@ -334,10 +308,8 @@ namespace ignite
                 static void SetConsoleHandler(ConsoleWriteHandler consoleHandler);
                 static int RemoveConsoleHandler(ConsoleWriteHandler consoleHandler);
 
-                jobject IgnitionStart(char* cfgPath, char* name, int factoryId, long long dataPtr);
-                jobject IgnitionStart(char* cfgPath, char* name, int factoryId, long long dataPtr, JniErrorInfo* errInfo);
-                jobject IgnitionInstance(char* name);
-                jobject IgnitionInstance(char* name, JniErrorInfo* errInfo);
+                void IgnitionStart(char* cfgPath, char* name, int factoryId, long long dataPtr);
+                void IgnitionStart(char* cfgPath, char* name, int factoryId, long long dataPtr, JniErrorInfo* errInfo);
                 long long IgnitionEnvironmentPointer(char* name);
                 long long IgnitionEnvironmentPointer(char* name, JniErrorInfo* errInfo);
                 bool IgnitionStop(char* name, bool cancel);
@@ -345,41 +317,6 @@ namespace ignite
                 void IgnitionStopAll(bool cancel);
                 void IgnitionStopAll(bool cancel, JniErrorInfo* errInfo);
                 
-                void ProcessorReleaseStart(jobject obj);
-                jobject ProcessorProjection(jobject obj, JniErrorInfo* errInfo = NULL);
-                jobject ProcessorCache(jobject obj, const char* name);
-                jobject ProcessorCache(jobject obj, const char* name, JniErrorInfo* errInfo);
-                jobject ProcessorCreateCache(jobject obj, const char* name);
-                jobject ProcessorCreateCache(jobject obj, const char* name, JniErrorInfo* errInfo);
-                jobject ProcessorGetOrCreateCache(jobject obj, const char* name);
-                jobject ProcessorGetOrCreateCache(jobject obj, const char* name, JniErrorInfo* errInfo);
-                jobject ProcessorCreateCacheFromConfig(jobject obj, long long memPtr);
-                jobject ProcessorCreateCacheFromConfig(jobject obj, long long memPtr, JniErrorInfo* errInfo);
-                jobject ProcessorGetOrCreateCacheFromConfig(jobject obj, long long memPtr);
-                jobject ProcessorGetOrCreateCacheFromConfig(jobject obj, long long memPtr, JniErrorInfo* errInfo);
-                jobject ProcessorCreateNearCache(jobject obj, const char* name, long long memPtr);
-                jobject ProcessorGetOrCreateNearCache(jobject obj, const char* name, long long memPtr);
-                void ProcessorDestroyCache(jobject obj, const char* name);
-                void ProcessorDestroyCache(jobject obj, const char* name, JniErrorInfo* errInfo);
-                jobject ProcessorAffinity(jobject obj, const char* name);
-                jobject ProcessorDataStreamer(jobject obj, const char* name, bool keepPortable);
-                jobject ProcessorTransactions(jobject obj, JniErrorInfo* errInfo = NULL);
-                jobject ProcessorCompute(jobject obj, jobject prj);
-                jobject ProcessorCompute(jobject obj, jobject prj, JniErrorInfo* errInfo);
-                jobject ProcessorMessage(jobject obj, jobject prj);
-                jobject ProcessorEvents(jobject obj, jobject prj);
-                jobject ProcessorServices(jobject obj, jobject prj);
-                jobject ProcessorExtensions(jobject obj);
-                jobject ProcessorExtension(jobject obj, int id);
-                jobject ProcessorAtomicLong(jobject obj, char* name, long long initVal, bool create);
-                jobject ProcessorAtomicSequence(jobject obj, char* name, long long initVal, bool create);
-                jobject ProcessorAtomicReference(jobject obj, char* name, long long memPtr, bool create);
-				void ProcessorGetIgniteConfiguration(jobject obj, long long memPtr);
-				void ProcessorGetCacheNames(jobject obj, long long memPtr);
-				bool ProcessorLoggerIsLevelEnabled(jobject obj, int level);
-				void ProcessorLoggerLog(jobject obj, int level, char* message, char* category, char* errorInfo);
-                jobject ProcessorBinaryProcessor(jobject obj);
-
                 long long TargetInLongOutLong(jobject obj, int type, long long memPtr, JniErrorInfo* errInfo = NULL);
                 long long TargetInStreamOutLong(jobject obj, int type, long long memPtr, JniErrorInfo* errInfo = NULL);
                 void TargetInStreamOutStream(jobject obj, int opType, long long inMemPtr, long long outMemPtr, JniErrorInfo* errInfo = NULL);
@@ -406,9 +343,6 @@ namespace ignite
                 void ExceptionCheck(JNIEnv* env);
                 void ExceptionCheck(JNIEnv* env, JniErrorInfo* errInfo);
                 jobject LocalToGlobal(JNIEnv* env, jobject obj);
-                jobject ProcessorCache0(jobject proc, const char* name, jmethodID mthd, JniErrorInfo* errInfo);
-                jobject ProcessorCacheFromConfig0(jobject proc, long long memPtr, jmethodID mthd, JniErrorInfo* errInfo);
-                jobject ProcessorGetOrCreateNearCache0(jobject obj, const char* name, long long memPtr, jmethodID methodID);
             };
 
             JNIEXPORT jlong JNICALL JniCacheStoreCreate(JNIEnv *env, jclass cls, jlong envPtr, jlong memPtr);

http://git-wip-us.apache.org/repos/asf/ignite/blob/bdc9e4b9/modules/platforms/cpp/jni/project/vs/module.def
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/jni/project/vs/module.def b/modules/platforms/cpp/jni/project/vs/module.def
index 82cc41e..53e7e42 100644
--- a/modules/platforms/cpp/jni/project/vs/module.def
+++ b/modules/platforms/cpp/jni/project/vs/module.def
@@ -2,19 +2,9 @@ LIBRARY ignite.jni.dll
 EXPORTS
 IgniteReallocate @1 
 IgniteIgnitionStart @2 
-IgniteIgnitionInstance @3 
 IgniteIgnitionEnvironmentPointer @4 
 IgniteIgnitionStop @5 
 IgniteIgnitionStopAll @6 
-IgniteProcessorReleaseStart @8 
-IgniteProcessorProjection @9 
-IgniteProcessorCache @10 
-IgniteProcessorCreateCache @11 
-IgniteProcessorGetOrCreateCache @12 
-IgniteProcessorAffinity @13 
-IgniteProcessorDataStreamer @14 
-IgniteProcessorTransactions @15 
-IgniteProcessorServices @16
 IgniteTargetInStreamOutObject @17 
 IgniteTargetInStreamOutLong @18 
 IgniteTargetOutStream @19 
@@ -22,9 +12,6 @@ IgniteTargetInStreamOutStream @20
 IgniteTargetInObjectStreamOutObjectStream @21
 IgniteTargetInLongOutLong @24
 IgniteTargetInStreamAsync @25
-IgniteProcessorCompute @64 
-IgniteProcessorMessage @65 
-IgniteProcessorEvents @66 
 IgniteAcquire @80
 IgniteRelease @81
 IgniteThrowToJava @82 
@@ -33,20 +20,5 @@ IgniteCreateContext @84
 IgniteDeleteContext @85 
 IgniteDestroyJvm @86 
 IgniteTargetOutObject @91 
-IgniteProcessorExtension @96
-IgniteProcessorExtensions @97
-IgniteProcessorAtomicLong @98
-IgniteProcessorCreateCacheFromConfig @114
-IgniteProcessorGetOrCreateCacheFromConfig @115
-IgniteProcessorGetIgniteConfiguration @116
-IgniteProcessorDestroyCache @117
-IgniteProcessorAtomicSequence @118
-IgniteProcessorAtomicReference @128
-IgniteProcessorCreateNearCache @131
-IgniteProcessorGetOrCreateNearCache @132
-IgniteProcessorGetCacheNames @133
 IgniteSetConsoleHandler @135
-IgniteRemoveConsoleHandler @136
-IgniteProcessorLoggerIsLevelEnabled @137
-IgniteProcessorLoggerLog @138
-IgniteProcessorBinaryProcessor @139
\ No newline at end of file
+IgniteRemoveConsoleHandler @136
\ No newline at end of file