You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by pt...@apache.org on 2016/11/14 09:26:35 UTC

[23/33] ignite git commit: IGNITE-4028 Get rid of OP_META in PlatformAbstractTarget

IGNITE-4028 Get rid of OP_META in PlatformAbstractTarget

This closes #1192


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

Branch: refs/heads/master
Commit: ead15193899d08f41491166003cabed0560f0c59
Parents: 3485939
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Mon Nov 7 10:49:03 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Mon Nov 7 10:49:03 2016 +0300

----------------------------------------------------------------------
 .../platform/PlatformAbstractTarget.java        |  11 +-
 .../platform/PlatformNoopProcessor.java         |   5 +
 .../processors/platform/PlatformProcessor.java  |   7 +
 .../platform/PlatformProcessorImpl.java         |   6 +
 .../binary/PlatformBinaryProcessor.java         |  96 ++++++++++++
 .../platform/cluster/PlatformClusterGroup.java  |  31 ----
 .../ignite/impl/binary/binary_type_manager.h    |  14 +-
 .../impl/binary/binary_type_updater_impl.h      |  12 +-
 .../include/ignite/impl/ignite_environment.h    |  49 ++++--
 modules/platforms/cpp/core/src/ignition.cpp     |   8 +-
 .../impl/binary/binary_type_updater_impl.cpp    |  13 +-
 .../cpp/core/src/impl/ignite_environment.cpp    |  45 ++++--
 .../core/src/impl/interop/interop_target.cpp    |   4 +-
 .../cpp/jni/include/ignite/jni/exports.h        |   1 +
 .../platforms/cpp/jni/include/ignite/jni/java.h |   2 +
 .../cpp/jni/include/ignite/jni/utils.h          |  91 ++++++++++-
 modules/platforms/cpp/jni/project/vs/module.def |   1 +
 modules/platforms/cpp/jni/src/exports.cpp       |   4 +
 modules/platforms/cpp/jni/src/java.cpp          |  13 ++
 .../Apache.Ignite.Core.csproj                   |   2 +-
 .../Apache.Ignite.Core/Impl/Binary/Binary.cs    |   2 +-
 .../Impl/Binary/BinaryObjectSchemaSerializer.cs |   2 +-
 .../Impl/Binary/BinaryProcessor.cs              | 156 +++++++++++++++++++
 .../Impl/Binary/BinaryReader.cs                 |   2 +-
 .../Impl/Binary/Marshaller.cs                   |  11 +-
 .../Impl/Cluster/ClusterGroupImpl.cs            |  72 ++-------
 .../Impl/Cluster/IClusterGroupEx.cs             |  35 -----
 .../dotnet/Apache.Ignite.Core/Impl/Ignite.cs    |  29 ++--
 .../Apache.Ignite.Core/Impl/IgniteProxy.cs      |   9 +-
 .../Apache.Ignite.Core/Impl/PlatformTarget.cs   |  64 --------
 .../Impl/Unmanaged/IgniteJniNativeMethods.cs    |   3 +
 .../Impl/Unmanaged/UnmanagedUtils.cs            |   7 +
 32 files changed, 526 insertions(+), 281 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/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 cba0031..2df86ac 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
@@ -43,9 +43,6 @@ public abstract class PlatformAbstractTarget implements PlatformTarget {
     /** Constant: ERROR. */
     protected static final int ERROR = -1;
 
-    /** */
-    private static final int OP_META = -1;
-
     /** Context. */
     protected final PlatformContext platformCtx;
 
@@ -78,13 +75,7 @@ public abstract class PlatformAbstractTarget implements PlatformTarget {
         try (PlatformMemory mem = platformCtx.memory().get(memPtr)) {
             BinaryRawReaderEx reader = platformCtx.reader(mem);
 
-            if (type == OP_META) {
-                platformCtx.processMetadata(reader);
-
-                return TRUE;
-            }
-            else
-                return processInStreamOutLong(type, reader, mem);
+            return processInStreamOutLong(type, reader, mem);
         }
         catch (Exception e) {
             throw convertException(e);

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/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 a7b7a8d..fd357ec 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
@@ -185,4 +185,9 @@ public class PlatformNoopProcessor extends GridProcessorAdapter implements Platf
     @Override public void loggerLog(int level, String message, String category, String errorInfo) {
         // No-op.
     }
+
+    /** {@inheritDoc} */
+    @Override public PlatformTarget binaryProcessor() {
+        return null;
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/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 1d9d3cd..f01175e 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
@@ -271,4 +271,11 @@ public interface PlatformProcessor extends GridProcessor {
      * @param errorInfo Error info.
      */
     public void loggerLog(int level, String message, String category, String errorInfo);
+
+    /**
+     * Gets the binary processor.
+     *
+     * @return Binary processor.
+     */
+    public PlatformTarget binaryProcessor();
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/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 d875c7e..f775987 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
@@ -35,6 +35,7 @@ import org.apache.ignite.internal.processors.GridProcessorAdapter;
 import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
 import org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl;
 import org.apache.ignite.internal.processors.datastructures.GridCacheAtomicLongImpl;
+import org.apache.ignite.internal.processors.platform.binary.PlatformBinaryProcessor;
 import org.apache.ignite.internal.processors.platform.cache.PlatformCache;
 import org.apache.ignite.internal.processors.platform.cache.PlatformCacheExtension;
 import org.apache.ignite.internal.processors.platform.cache.affinity.PlatformAffinity;
@@ -502,6 +503,11 @@ public class PlatformProcessorImpl extends GridProcessorAdapter implements Platf
         }
     }
 
+    /** {@inheritDoc} */
+    @Override public PlatformTarget binaryProcessor() {
+        return new PlatformBinaryProcessor(platformCtx);
+    }
+
     /**
      * Gets the near cache config.
      *

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/binary/PlatformBinaryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/binary/PlatformBinaryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/binary/PlatformBinaryProcessor.java
new file mode 100644
index 0000000..1bb577e
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/binary/PlatformBinaryProcessor.java
@@ -0,0 +1,96 @@
+/*
+ * 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.platform.binary;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.binary.BinaryRawReaderEx;
+import org.apache.ignite.internal.binary.BinaryRawWriterEx;
+import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget;
+import org.apache.ignite.internal.processors.platform.PlatformContext;
+
+/**
+ * Platform binary processor.
+ */
+public class PlatformBinaryProcessor extends PlatformAbstractTarget {
+    /** */
+    private static final int OP_GET_META = 1;
+
+    /** */
+    private static final int OP_GET_ALL_META = 2;
+
+    /** */
+    private static final int OP_PUT_META = 3;
+
+    /** */
+    private static final int OP_GET_SCHEMA = 4;
+
+    /**
+     * Constructor.
+     *
+     * @param platformCtx Context.
+     */
+    public PlatformBinaryProcessor(PlatformContext platformCtx) {
+        super(platformCtx);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected long processInStreamOutLong(int type, BinaryRawReaderEx reader) throws IgniteCheckedException {
+        if (type == OP_PUT_META) {
+            platformCtx.processMetadata(reader);
+
+            return TRUE;
+        }
+
+        return super.processInStreamOutLong(type, reader);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void processOutStream(int type, BinaryRawWriterEx writer) throws IgniteCheckedException {
+        if (type == OP_GET_ALL_META)
+            platformCtx.writeAllMetadata(writer);
+        else
+            super.processOutStream(type, writer);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void processInStreamOutStream(int type, BinaryRawReaderEx reader,
+        BinaryRawWriterEx writer) throws IgniteCheckedException {
+        switch (type) {
+            case OP_GET_META: {
+                int typeId = reader.readInt();
+
+                platformCtx.writeMetadata(writer, typeId);
+
+                break;
+            }
+
+            case OP_GET_SCHEMA: {
+                int typeId = reader.readInt();
+                int schemaId = reader.readInt();
+
+                platformCtx.writeSchema(writer, typeId, schemaId);
+
+                break;
+            }
+
+            default:
+                super.processInStreamOutStream(type, reader, writer);
+                break;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/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 dde98f5..724eea0 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
@@ -38,9 +38,6 @@ import org.jetbrains.annotations.Nullable;
 @SuppressWarnings({"UnusedDeclaration"})
 public class PlatformClusterGroup extends PlatformAbstractTarget {
     /** */
-    private static final int OP_ALL_METADATA = 1;
-
-    /** */
     private static final int OP_FOR_ATTRIBUTE = 2;
 
     /** */
@@ -59,9 +56,6 @@ public class PlatformClusterGroup extends PlatformAbstractTarget {
     private static final int OP_FOR_NODE_IDS = 7;
 
     /** */
-    private static final int OP_METADATA = 8;
-
-    /** */
     private static final int OP_METRICS = 9;
 
     /** */
@@ -80,9 +74,6 @@ public class PlatformClusterGroup extends PlatformAbstractTarget {
     private static final int OP_TOPOLOGY = 14;
 
     /** */
-    private static final int OP_SCHEMA = 15;
-
-    /** */
     private static final int OP_FOR_OTHERS = 16;
 
     /** */
@@ -130,11 +121,6 @@ public class PlatformClusterGroup extends PlatformAbstractTarget {
 
                 break;
 
-            case OP_ALL_METADATA:
-                platformCtx.writeAllMetadata(writer);
-
-                break;
-
             default:
                 super.processOutStream(type, writer);
         }
@@ -201,14 +187,6 @@ public class PlatformClusterGroup extends PlatformAbstractTarget {
                 break;
             }
 
-            case OP_METADATA: {
-                int typeId = reader.readInt();
-
-                platformCtx.writeMetadata(writer, typeId);
-
-                break;
-            }
-
             case OP_TOPOLOGY: {
                 long topVer = reader.readLong();
 
@@ -217,15 +195,6 @@ public class PlatformClusterGroup extends PlatformAbstractTarget {
                 break;
             }
 
-            case OP_SCHEMA: {
-                int typeId = reader.readInt();
-                int schemaId = reader.readInt();
-
-                platformCtx.writeSchema(writer, typeId, schemaId);
-
-                break;
-            }
-
             default:
                 super.processInStreamOutStream(type, reader, writer);
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_type_manager.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_type_manager.h b/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_type_manager.h
index 6019a2c..8aca1a0 100644
--- a/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_type_manager.h
+++ b/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_type_manager.h
@@ -65,7 +65,7 @@ namespace ignite
                 /**
                  * Get current type manager version.
                  *
-                 * @param Version.
+                 * @return Version.
                  */
                 int32_t GetVersion();
 
@@ -80,7 +80,7 @@ namespace ignite
                 /**
                  * Process pending updates.
                  *
-                 * @param updated Updater.
+                 * @param updater Updater.
                  * @param err Error.
                  * @return In case of success.
                  */
@@ -89,18 +89,18 @@ namespace ignite
             private:
                 /** Current snapshots. */
                 ignite::common::concurrent::SharedPointer<std::map<int32_t, SPSnap>> snapshots;
-                
+
                 /** Pending snapshots. */
-                std::vector<SPSnap>* pending;                                          
+                std::vector<SPSnap>* pending;
 
                 /** Critical section. */
                 ignite::common::concurrent::CriticalSection* cs;
 
                 /** Version of pending changes. */
-                int32_t pendingVer;                                                    
-                
+                int32_t pendingVer;
+
                 /** Latest version. */
-                int32_t ver;          
+                int32_t ver;
 
                 IGNITE_NO_COPY_ASSIGNMENT(BinaryTypeManager);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/modules/platforms/cpp/core/include/ignite/impl/binary/binary_type_updater_impl.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/impl/binary/binary_type_updater_impl.h b/modules/platforms/cpp/core/include/ignite/impl/binary/binary_type_updater_impl.h
index 4edad4f..bd21751 100644
--- a/modules/platforms/cpp/core/include/ignite/impl/binary/binary_type_updater_impl.h
+++ b/modules/platforms/cpp/core/include/ignite/impl/binary/binary_type_updater_impl.h
@@ -24,7 +24,7 @@
 #include "ignite/impl/binary/binary_type_updater.h"
 
 namespace ignite
-{    
+{
     namespace impl
     {
         namespace binary
@@ -41,7 +41,7 @@ namespace ignite
                  * @param env Environment.
                  * @param javaRef Reference to Java object which is able to process type request.
                  */
-                BinaryTypeUpdaterImpl(ignite::common::concurrent::SharedPointer<IgniteEnvironment> env, jobject javaRef);
+                BinaryTypeUpdaterImpl(IgniteEnvironment& env, jobject javaRef);
 
                 /**
                  * Destructor.
@@ -51,15 +51,15 @@ namespace ignite
                 bool Update(Snap* snapshot, IgniteError* err);
             private:
                 /** Environment. */
-                ignite::common::concurrent::SharedPointer<IgniteEnvironment> env;
-                
+                IgniteEnvironment& env;
+
                 /** Handle to Java object. */
-                jobject javaRef;                 
+                jobject javaRef;
 
                 IGNITE_NO_COPY_ASSIGNMENT(BinaryTypeUpdaterImpl)
             };
         }
-    }    
+    }
 }
 
 #endif //_IGNITE_IMPL_BINARY_BINARY_TYPE_UPDATER_IMPL
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/modules/platforms/cpp/core/include/ignite/impl/ignite_environment.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/impl/ignite_environment.h b/modules/platforms/cpp/core/include/ignite/impl/ignite_environment.h
index d2486bb..fb6f657 100644
--- a/modules/platforms/cpp/core/include/ignite/impl/ignite_environment.h
+++ b/modules/platforms/cpp/core/include/ignite/impl/ignite_environment.h
@@ -23,6 +23,7 @@
 
 #include "ignite/impl/interop/interop_memory.h"
 #include "ignite/impl/binary/binary_type_manager.h"
+#include "ignite/jni/utils.h"
 
 namespace ignite 
 {
@@ -52,24 +53,29 @@ namespace ignite
             /**
              * Populate callback handlers.
              *
-             * @param Target (current env wrapped into a shared pointer).
+             * @param target (current env wrapped into a shared pointer).
              * @return JNI handlers.
              */
-            ignite::jni::java::JniHandlers GetJniHandlers(ignite::common::concurrent::SharedPointer<IgniteEnvironment>* target);
+            jni::java::JniHandlers GetJniHandlers(common::concurrent::SharedPointer<IgniteEnvironment>* target);
 
             /**
-             * Perform initialization on successful start.
+             * Set context.
              *
              * @param ctx Context.
              */
-            void Initialize(ignite::common::concurrent::SharedPointer<ignite::jni::java::JniContext> ctx);
+            void SetContext(common::concurrent::SharedPointer<jni::java::JniContext> ctx);
+
+            /**
+             * Perform initialization on successful start.
+             */
+            void Initialize();
 
             /**
              * Start callback.
              *
              * @param memPtr Memory pointer.
              */
-            void OnStartCallback(long long memPtr);
+            void OnStartCallback(long long memPtr, jobject proc);
 
             /**
              * Get name of Ignite instance.
@@ -83,14 +89,14 @@ namespace ignite
              *
              * @return Context.
              */
-            ignite::jni::java::JniContext* Context();
+            jni::java::JniContext* Context();
 
             /**
              * Get memory for interop operations.
              *
              * @return Memory.
              */
-            ignite::common::concurrent::SharedPointer<interop::InteropMemory> AllocateMemory();
+            common::concurrent::SharedPointer<interop::InteropMemory> AllocateMemory();
 
             /**
              * Get memory chunk for interop operations with desired capacity.
@@ -98,7 +104,7 @@ namespace ignite
              * @param cap Capacity.
              * @return Memory.
              */
-            ignite::common::concurrent::SharedPointer<interop::InteropMemory> AllocateMemory(int32_t cap);
+            common::concurrent::SharedPointer<interop::InteropMemory> AllocateMemory(int32_t cap);
 
             /**
              * Get memory chunk located at the given pointer.
@@ -106,27 +112,46 @@ namespace ignite
              * @param memPtr Memory pointer.
              * @retrun Memory.
              */
-            ignite::common::concurrent::SharedPointer<interop::InteropMemory> GetMemory(int64_t memPtr);
+            common::concurrent::SharedPointer<interop::InteropMemory> GetMemory(int64_t memPtr);
 
             /**
              * Get type manager.
              *
-             * @param Type manager.
+             * @return Type manager.
              */
             binary::BinaryTypeManager* GetTypeManager();
+
+            /**
+             * Get type updater.
+             *
+             * @return Type updater.
+             */
+            binary::BinaryTypeUpdater* GetTypeUpdater();
+
+            /**
+             * Notify processor that Ignite instance has started.
+             */
+            void ProcessorReleaseStart();
+
         private:
             /** Context to access Java. */
-            ignite::common::concurrent::SharedPointer<ignite::jni::java::JniContext> ctx;
+            common::concurrent::SharedPointer<jni::java::JniContext> ctx;
 
             /** Startup latch. */
-            ignite::common::concurrent::SingleLatch* latch;
+            common::concurrent::SingleLatch* latch;
 
             /** Ignite name. */
             char* name;
 
+            /** Processor instance. */
+            jni::JavaGlobalRef proc;
+
             /** Type manager. */
             binary::BinaryTypeManager* metaMgr;
 
+            /** Type updater. */
+            binary::BinaryTypeUpdater* metaUpdater;
+
             IGNITE_NO_COPY_ASSIGNMENT(IgniteEnvironment);
         };
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/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 83adb4c..72fbf7a 100644
--- a/modules/platforms/cpp/core/src/ignition.cpp
+++ b/modules/platforms/cpp/core/src/ignition.cpp
@@ -230,10 +230,12 @@ namespace ignite
                 if (!ctx.Get())
                 {
                     IgniteError::SetError(jniErr.code, jniErr.errCls, jniErr.errMsg, err);
-                    
+
                     failed = true;
                 }
 
+                env.Get()->SetContext(ctx);
+
                 // 5. Start Ignite.
                 if (!failed)
                 {
@@ -263,7 +265,7 @@ namespace ignite
                     }
                     else {
                         // 6. Ignite is started at this point.
-                        env.Get()->Initialize(ctx);
+                        env.Get()->Initialize();
 
                         started = true;
                     }
@@ -288,6 +290,8 @@ namespace ignite
         }
         else 
         {
+            env.Get()->ProcessorReleaseStart();
+
             IgniteImpl* impl = new IgniteImpl(env, javaRef);
 
             return Ignite(impl);

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/modules/platforms/cpp/core/src/impl/binary/binary_type_updater_impl.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/src/impl/binary/binary_type_updater_impl.cpp b/modules/platforms/cpp/core/src/impl/binary/binary_type_updater_impl.cpp
index e6375a6..ff6df9e 100644
--- a/modules/platforms/cpp/core/src/impl/binary/binary_type_updater_impl.cpp
+++ b/modules/platforms/cpp/core/src/impl/binary/binary_type_updater_impl.cpp
@@ -34,24 +34,25 @@ namespace ignite
         namespace binary
         {
             /** Operation: metadata update. */
-            const int32_t OP_METADATA = -1;
+            const int32_t OP_PUT_META = 3;
 
-            BinaryTypeUpdaterImpl::BinaryTypeUpdaterImpl(SharedPointer<IgniteEnvironment> env,
-                jobject javaRef) :  env(env), javaRef(javaRef)
+            BinaryTypeUpdaterImpl::BinaryTypeUpdaterImpl(IgniteEnvironment& env, jobject javaRef) :
+                env(env),
+                javaRef(javaRef)
             {
                 // No-op.
             }
 
             BinaryTypeUpdaterImpl::~BinaryTypeUpdaterImpl()
             {
-                // No-op.
+                JniContext::Release(javaRef);
             }
 
             bool BinaryTypeUpdaterImpl::Update(Snap* snap, IgniteError* err)
             {
                 JniErrorInfo jniErr;
 
-                SharedPointer<InteropMemory> mem = env.Get()->AllocateMemory();
+                SharedPointer<InteropMemory> mem = env.AllocateMemory();
 
                 InteropOutputStream out(mem.Get());
                 BinaryWriterImpl writer(&out, NULL);
@@ -85,7 +86,7 @@ namespace ignite
 
                 out.Synchronize();
 
-                long long res = env.Get()->Context()->TargetInStreamOutLong(javaRef, OP_METADATA, mem.Get()->PointerLong(), &jniErr);
+                long long res = env.Context()->TargetInStreamOutLong(javaRef, OP_PUT_META, mem.Get()->PointerLong(), &jniErr);
 
                 IgniteError::SetError(jniErr.code, jniErr.errCls, jniErr.errMsg, err);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/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 ab50866..32c855b 100644
--- a/modules/platforms/cpp/core/src/impl/ignite_environment.cpp
+++ b/modules/platforms/cpp/core/src/impl/ignite_environment.cpp
@@ -19,6 +19,7 @@
 #include "ignite/impl/binary/binary_reader_impl.h"
 #include "ignite/impl/ignite_environment.h"
 #include "ignite/binary/binary.h"
+#include "ignite/impl/binary/binary_type_updater_impl.h"
 
 using namespace ignite::common::concurrent;
 using namespace ignite::jni::java;
@@ -34,14 +35,14 @@ namespace ignite
          * OnStart callback.
          *
          * @param target Target environment.
-         * @param proc Processor instance (not used for now).
+         * @param proc Processor instance.
          * @param memPtr Memory pointer.
          */
         void IGNITE_CALL OnStart(void* target, void* proc, long long memPtr)
         {
             SharedPointer<IgniteEnvironment>* ptr = static_cast<SharedPointer<IgniteEnvironment>*>(target);
 
-            ptr->Get()->OnStartCallback(memPtr);
+            ptr->Get()->OnStartCallback(memPtr, reinterpret_cast<jobject>(proc));
         }
 
         /**
@@ -72,8 +73,8 @@ namespace ignite
             mem.Get()->Reallocate(cap);
         }
 
-        IgniteEnvironment::IgniteEnvironment() : ctx(SharedPointer<JniContext>()), latch(new SingleLatch), name(NULL),
-            metaMgr(new BinaryTypeManager())
+        IgniteEnvironment::IgniteEnvironment() : ctx(SharedPointer<JniContext>()), latch(new SingleLatch), name(0),
+            proc(), metaMgr(new BinaryTypeManager()), metaUpdater(0)
         {
             // No-op.
         }
@@ -81,11 +82,9 @@ namespace ignite
         IgniteEnvironment::~IgniteEnvironment()
         {
             delete latch;
-
-            if (name)
-                delete name;
-
+            delete name;
             delete metaMgr;
+            delete metaUpdater;
         }
 
         JniHandlers IgniteEnvironment::GetJniHandlers(SharedPointer<IgniteEnvironment>* target)
@@ -99,16 +98,23 @@ namespace ignite
 
             hnds.memRealloc = MemoryReallocate;
 
-            hnds.error = NULL;
+            hnds.error = 0;
 
             return hnds;
         }
 
-        void IgniteEnvironment::Initialize(SharedPointer<JniContext> ctx)
+        void IgniteEnvironment::SetContext(SharedPointer<JniContext> ctx)
         {
             this->ctx = ctx;
+        }
 
+        void IgniteEnvironment::Initialize()
+        {
             latch->CountDown();
+
+            jobject binaryProc = Context()->ProcessorBinaryProcessor(proc.Get());
+
+            metaUpdater = new BinaryTypeUpdaterImpl(*this, binaryProc);
         }
 
         const char* IgniteEnvironment::InstanceName() const
@@ -160,14 +166,27 @@ namespace ignite
             return metaMgr;
         }
 
-        void IgniteEnvironment::OnStartCallback(long long memPtr)
+        BinaryTypeUpdater* IgniteEnvironment::GetTypeUpdater()
         {
+            return metaUpdater;
+        }
+
+        void IgniteEnvironment::ProcessorReleaseStart()
+        {
+            if (proc.Get())
+                ctx.Get()->ProcessorReleaseStart(proc.Get());
+        }
+
+        void IgniteEnvironment::OnStartCallback(long long memPtr, jobject proc)
+        {
+            this->proc = jni::JavaGlobalRef(*ctx.Get(), proc);
+
             InteropExternalMemory mem(reinterpret_cast<int8_t*>(memPtr));
             InteropInputStream stream(&mem);
 
             BinaryReaderImpl reader(&stream);
 
-            int32_t nameLen = reader.ReadString(NULL, 0);
+            int32_t nameLen = reader.ReadString(0, 0);
 
             if (nameLen >= 0)
             {
@@ -175,7 +194,7 @@ namespace ignite
                 reader.ReadString(name, nameLen + 1);
             }
             else
-                name = NULL;
+                name = 0;
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/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 196c3f6..4992ccb 100644
--- a/modules/platforms/cpp/core/src/impl/interop/interop_target.cpp
+++ b/modules/platforms/cpp/core/src/impl/interop/interop_target.cpp
@@ -72,9 +72,7 @@ namespace ignite
 
                 if (metaMgr->IsUpdatedSince(metaVer))
                 {
-                    BinaryTypeUpdaterImpl metaUpdater(env, javaRef);
-
-                    if (!metaMgr->ProcessPendingUpdates(&metaUpdater, err))
+                    if (!metaMgr->ProcessPendingUpdates(env.Get()->GetTypeUpdater(), err))
                         return 0;
                 }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/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 3a98eda..3052435 100644
--- a/modules/platforms/cpp/jni/include/ignite/jni/exports.h
+++ b/modules/platforms/cpp/jni/include/ignite/jni/exports.h
@@ -56,6 +56,7 @@ extern "C" {
     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);

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/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 9dddd8c..07df001 100644
--- a/modules/platforms/cpp/jni/include/ignite/jni/java.h
+++ b/modules/platforms/cpp/jni/include/ignite/jni/java.h
@@ -269,6 +269,7 @@ namespace ignite
                 jmethodID m_PlatformProcessor_atomicReference;
                 jmethodID m_PlatformProcessor_loggerIsLevelEnabled;
                 jmethodID m_PlatformProcessor_loggerLog;
+                jmethodID m_PlatformProcessor_binaryProcessor;
 
                 jclass c_PlatformTarget;
                 jmethodID m_PlatformTarget_inLongOutLong;
@@ -449,6 +450,7 @@ namespace ignite
 				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);

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/modules/platforms/cpp/jni/include/ignite/jni/utils.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/jni/include/ignite/jni/utils.h b/modules/platforms/cpp/jni/include/ignite/jni/utils.h
index ee40248..2b22cf9 100644
--- a/modules/platforms/cpp/jni/include/ignite/jni/utils.h
+++ b/modules/platforms/cpp/jni/include/ignite/jni/utils.h
@@ -19,6 +19,7 @@
 
 #include <string>
 
+#include <ignite/jni/java.h>
 #include <ignite/common/common.h>
 
 namespace ignite
@@ -35,7 +36,7 @@ namespace ignite
              * Destructor.
              */
             ~AttachHelper();
-			
+
             /**
              * Callback invoked on successful thread attach ot JVM.
              */
@@ -43,6 +44,94 @@ namespace ignite
         };
 
         /**
+        * Represents global reference to Java object.
+        */
+        class IGNITE_IMPORT_EXPORT JavaGlobalRef
+        {
+        public:
+            /**
+            * Default constructor
+            */
+            JavaGlobalRef() :
+                ctx(0),
+                obj(0)
+            {
+                // No-op.
+            }
+
+            /**
+            * Constructor
+            *
+            * @param ctx JNI context.
+            * @param obj Java object.
+            */
+            JavaGlobalRef(java::JniContext& ctx, jobject obj) :
+                ctx(&ctx),
+                obj(ctx.Acquire(obj))
+            {
+                // No-op.
+            }
+
+            /**
+            * Copy constructor
+            *
+            * @param other Other instance.
+            */
+            JavaGlobalRef(const JavaGlobalRef& other) :
+                ctx(other.ctx),
+                obj(ctx->Acquire(other.obj))
+            {
+                // No-op.
+            }
+
+            /**
+            * Assignment operator.
+            *
+            * @param other Other instance.
+            * @return *this.
+            */
+            JavaGlobalRef& operator=(const JavaGlobalRef& other)
+            {
+                if (this != &other)
+                {
+                    if (ctx)
+                        ctx->Release(obj);
+
+                    ctx = other.ctx;
+                    obj = ctx->Acquire(other.obj);
+                }
+
+                return *this;
+            }
+
+            /**
+            * Destructor.
+            */
+            ~JavaGlobalRef()
+            {
+                if (ctx)
+                    ctx->Release(obj);
+            }
+
+            /**
+            * Get object.
+            *
+            * @return Object.
+            */
+            jobject Get()
+            {
+                return obj;
+            }
+
+        private:
+            /** Context. */
+            java::JniContext* ctx;
+
+            /** Object. */
+            jobject obj;
+        };
+
+        /**
          * Attempts to find JVM library to load it into the process later.
          * First search is performed using the passed path argument (is not NULL).
          * Then JRE_HOME is evaluated. Last, JAVA_HOME is evaluated.

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/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 258e80c..e58ac3b 100644
--- a/modules/platforms/cpp/jni/project/vs/module.def
+++ b/modules/platforms/cpp/jni/project/vs/module.def
@@ -51,3 +51,4 @@ IgniteSetConsoleHandler @135
 IgniteRemoveConsoleHandler @136
 IgniteProcessorLoggerIsLevelEnabled @137
 IgniteProcessorLoggerLog @138
+IgniteProcessorBinaryProcessor @139
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/modules/platforms/cpp/jni/src/exports.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/jni/src/exports.cpp b/modules/platforms/cpp/jni/src/exports.cpp
index e87cbd3..dde98fb 100644
--- a/modules/platforms/cpp/jni/src/exports.cpp
+++ b/modules/platforms/cpp/jni/src/exports.cpp
@@ -150,6 +150,10 @@ extern "C" {
         ctx->ProcessorLoggerLog(static_cast<jobject>(obj), level, message, category, errorInfo);
     }
 
+    void* IGNITE_CALL IgniteProcessorBinaryProcessor(gcj::JniContext* ctx, void* obj) {
+        return ctx->ProcessorBinaryProcessor(static_cast<jobject>(obj));
+    }
+
     long long IGNITE_CALL IgniteTargetInStreamOutLong(gcj::JniContext* ctx, void* obj, int opType, long long memPtr) {
         return ctx->TargetInStreamOutLong(static_cast<jobject>(obj), opType, memPtr);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/modules/platforms/cpp/jni/src/java.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/jni/src/java.cpp b/modules/platforms/cpp/jni/src/java.cpp
index e2c9bf7..2d3cf72 100644
--- a/modules/platforms/cpp/jni/src/java.cpp
+++ b/modules/platforms/cpp/jni/src/java.cpp
@@ -218,6 +218,7 @@ namespace ignite
             JniMethod M_PLATFORM_PROCESSOR_GET_CACHE_NAMES = JniMethod("getCacheNames", "(J)V", false);
             JniMethod M_PLATFORM_PROCESSOR_LOGGER_IS_LEVEL_ENABLED = JniMethod("loggerIsLevelEnabled", "(I)Z", false);
             JniMethod M_PLATFORM_PROCESSOR_LOGGER_LOG = JniMethod("loggerLog", "(ILjava/lang/String;Ljava/lang/String;Ljava/lang/String;)V", false);
+            JniMethod M_PLATFORM_PROCESSOR_BINARY_PROCESSOR = JniMethod("binaryProcessor", "()Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
 
             const char* C_PLATFORM_TARGET = "org/apache/ignite/internal/processors/platform/PlatformTarget";
             JniMethod M_PLATFORM_TARGET_IN_LONG_OUT_LONG = JniMethod("inLongOutLong", "(IJ)J", false);
@@ -534,6 +535,7 @@ namespace ignite
 				m_PlatformProcessor_getCacheNames = FindMethod(env, c_PlatformProcessor, M_PLATFORM_PROCESSOR_GET_CACHE_NAMES);
 				m_PlatformProcessor_loggerIsLevelEnabled = FindMethod(env, c_PlatformProcessor, M_PLATFORM_PROCESSOR_LOGGER_IS_LEVEL_ENABLED);
 				m_PlatformProcessor_loggerLog = FindMethod(env, c_PlatformProcessor, M_PLATFORM_PROCESSOR_LOGGER_LOG);
+				m_PlatformProcessor_binaryProcessor = FindMethod(env, c_PlatformProcessor, M_PLATFORM_PROCESSOR_BINARY_PROCESSOR);
 
                 c_PlatformTarget = FindClass(env, C_PLATFORM_TARGET);
                 m_PlatformTarget_inLongOutLong = FindMethod(env, c_PlatformTarget, M_PLATFORM_TARGET_IN_LONG_OUT_LONG);
@@ -1339,6 +1341,17 @@ namespace ignite
                 ExceptionCheck(env);
             }
 
+            jobject JniContext::ProcessorBinaryProcessor(jobject obj)
+            {
+                JNIEnv* env = Attach();
+
+                jobject res = env->CallObjectMethod(obj, jvm->GetMembers().m_PlatformProcessor_binaryProcessor);
+
+                ExceptionCheck(env);
+
+                return LocalToGlobal(env, res);
+            }
+
             long long JniContext::TargetInStreamOutLong(jobject obj, int opType, long long memPtr, JniErrorInfo* err) {
                 JNIEnv* env = Attach();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
index 8e8f8ca..2973bb3 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
@@ -91,6 +91,7 @@
     <Compile Include="Discovery\Tcp\Multicast\Package-Info.cs" />
     <Compile Include="Discovery\Tcp\Package-Info.cs" />
     <Compile Include="Discovery\Tcp\Static\Package-Info.cs" />
+    <Compile Include="Impl\Binary\BinaryProcessor.cs" />
     <Compile Include="Impl\Binary\BinaryReflectiveSerializerInternal.cs" />
     <Compile Include="Impl\Binary\IBinarySerializerInternal.cs" />
     <Compile Include="Binary\Package-Info.cs" />
@@ -287,7 +288,6 @@
     <Compile Include="Impl\Cluster\ClusterGroupImpl.cs" />
     <Compile Include="Impl\Cluster\ClusterMetricsImpl.cs" />
     <Compile Include="Impl\Cluster\ClusterNodeImpl.cs" />
-    <Compile Include="Impl\Cluster\IClusterGroupEx.cs" />
     <Compile Include="Impl\Collections\CollectionExtensions.cs" />
     <Compile Include="Impl\Collections\MultiValueDictionary.cs" />
     <Compile Include="Impl\Collections\ReadOnlyCollection.cs" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Binary.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Binary.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Binary.cs
index 1deac07..3d55acd 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Binary.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Binary.cs
@@ -120,7 +120,7 @@ namespace Apache.Ignite.Core.Impl.Binary
         /** <inheritDoc /> */
         public ICollection<IBinaryType> GetBinaryTypes()
         {
-            return Marshaller.Ignite.ClusterGroup.GetBinaryTypes();
+            return Marshaller.Ignite.BinaryProcessor.GetBinaryTypes();
         }
 
         /** <inheritDoc /> */

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchemaSerializer.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchemaSerializer.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchemaSerializer.cs
index da86c07..f1d2f6a 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchemaSerializer.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchemaSerializer.cs
@@ -250,7 +250,7 @@ namespace Apache.Ignite.Core.Impl.Binary
             if (fieldIds == null)
             {
                 if (marsh.Ignite != null)
-                    fieldIds = marsh.Ignite.ClusterGroup.GetSchema(hdr.TypeId, hdr.SchemaId);
+                    fieldIds = marsh.Ignite.BinaryProcessor.GetSchema(hdr.TypeId, hdr.SchemaId);
 
                 if (fieldIds == null)
                     throw new BinaryObjectException("Cannot find schema for object with compact footer [" +

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryProcessor.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryProcessor.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryProcessor.cs
new file mode 100644
index 0000000..6935fa2
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryProcessor.cs
@@ -0,0 +1,156 @@
+\ufeff/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Impl.Binary
+{
+    using System.Collections.Generic;
+    using System.Diagnostics;
+    using Apache.Ignite.Core.Binary;
+    using Apache.Ignite.Core.Impl.Binary.Metadata;
+    using Apache.Ignite.Core.Impl.Unmanaged;
+
+    /// <summary>
+    /// Binary metadata processor.
+    /// </summary>
+    internal class BinaryProcessor : PlatformTarget
+    {
+        /// <summary>
+        /// Op codes.
+        /// </summary>
+        private enum Op
+        {
+            GetMeta = 1,
+            GetAllMeta = 2,
+            PutMeta = 3,
+            GetSchema = 4
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="BinaryProcessor"/> class.
+        /// </summary>
+        /// <param name="target">Target.</param>
+        /// <param name="marsh">Marshaller.</param>
+        public BinaryProcessor(IUnmanagedTarget target, Marshaller marsh) : base(target, marsh)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Gets metadata for specified type.
+        /// </summary>
+        public IBinaryType GetBinaryType(int typeId)
+        {
+            return DoOutInOp<IBinaryType>((int) Op.GetMeta,
+                writer => writer.WriteInt(typeId),
+                stream =>
+                {
+                    var reader = Marshaller.StartUnmarshal(stream, false);
+
+                    return reader.ReadBoolean() ? new BinaryType(reader) : null;
+                }
+            );
+        }
+
+        /// <summary>
+        /// Gets metadata for all known types.
+        /// </summary>
+        public List<IBinaryType> GetBinaryTypes()
+        {
+            return DoInOp((int) Op.GetAllMeta, s =>
+            {
+                var reader = Marshaller.StartUnmarshal(s);
+
+                var size = reader.ReadInt();
+
+                var res = new List<IBinaryType>(size);
+
+                for (var i = 0; i < size; i++)
+                    res.Add(reader.ReadBoolean() ? new BinaryType(reader) : null);
+
+                return res;
+            });
+        }
+
+        /// <summary>
+        /// Gets the schema.
+        /// </summary>
+        public int[] GetSchema(int typeId, int schemaId)
+        {
+            return DoOutInOp<int[]>((int) Op.GetSchema, writer =>
+            {
+                writer.WriteInt(typeId);
+                writer.WriteInt(schemaId);
+            });
+        }
+
+        /// <summary>
+        /// Put binary types to Grid.
+        /// </summary>
+        /// <param name="types">Binary types.</param>
+        internal void PutBinaryTypes(ICollection<BinaryType> types)
+        {
+            DoOutOp((int) Op.PutMeta, w =>
+            {
+                w.WriteInt(types.Count);
+
+                foreach (var meta in types)
+                {
+                    w.WriteInt(meta.TypeId);
+                    w.WriteString(meta.TypeName);
+                    w.WriteString(meta.AffinityKeyFieldName);
+
+                    var fields = meta.GetFieldsMap();
+
+                    w.WriteInt(fields.Count);
+
+                    foreach (var field in fields)
+                    {
+                        w.WriteString(field.Key);
+                        w.WriteInt(field.Value);
+                    }
+
+                    w.WriteBoolean(meta.IsEnum);
+
+                    // Send schemas
+                    var desc = meta.Descriptor;
+                    Debug.Assert(desc != null);
+
+                    var count = 0;
+                    var countPos = w.Stream.Position;
+                    w.WriteInt(0); // Reserve for count
+
+                    foreach (var schema in desc.Schema.GetAll())
+                    {
+                        w.WriteInt(schema.Key);
+
+                        var ids = schema.Value;
+                        w.WriteInt(ids.Length);
+
+                        foreach (var id in ids)
+                            w.WriteInt(id);
+
+                        count++;
+                    }
+
+                    w.Stream.WriteInt(countPos, count);
+                }
+            });
+
+            Marshaller.OnBinaryTypesSent(types);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReader.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReader.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReader.cs
index f048e97..4c34f73 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReader.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReader.cs
@@ -768,7 +768,7 @@ namespace Apache.Ignite.Core.Impl.Binary
             if (_curHdr.IsCompactFooter)
             {
                 // Get schema from Java
-                var schema = Marshaller.Ignite.ClusterGroup.GetSchema(_curHdr.TypeId, _curHdr.SchemaId);
+                var schema = Marshaller.Ignite.BinaryProcessor.GetSchema(_curHdr.TypeId, _curHdr.SchemaId);
 
                 if (schema == null)
                     throw new BinaryObjectException("Cannot find schema for object with compact footer [" +

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Marshaller.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Marshaller.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Marshaller.cs
index 6e63e9a..7acdfaa 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Marshaller.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Marshaller.cs
@@ -161,7 +161,9 @@ namespace Apache.Ignite.Core.Impl.Binary
             var ignite = Ignite;
 
             if (ignite != null && metas != null && metas.Count > 0)
-                ignite.PutBinaryTypes(metas);
+            {
+                ignite.BinaryProcessor.PutBinaryTypes(metas);
+            }
         }
 
         /// <summary>
@@ -270,7 +272,7 @@ namespace Apache.Ignite.Core.Impl.Binary
         {
             if (Ignite != null)
             {
-                IBinaryType meta = Ignite.GetBinaryType(typeId);
+                IBinaryType meta = Ignite.BinaryProcessor.GetBinaryType(typeId);
 
                 if (meta != null)
                     return meta;
@@ -290,7 +292,10 @@ namespace Apache.Ignite.Core.Impl.Binary
             GetBinaryTypeHandler(desc);  // ensure that handler exists
 
             if (Ignite != null)
-                Ignite.PutBinaryTypes(new[] {new BinaryType(desc)});
+            {
+                ICollection<BinaryType> metas = new[] {new BinaryType(desc)};
+                Ignite.BinaryProcessor.PutBinaryTypes(metas);
+            }
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs
index 277d61f..6c8779c 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs
@@ -29,7 +29,6 @@ namespace Apache.Ignite.Core.Impl.Cluster
     using Apache.Ignite.Core.Compute;
     using Apache.Ignite.Core.Events;
     using Apache.Ignite.Core.Impl.Binary;
-    using Apache.Ignite.Core.Impl.Binary.Metadata;
     using Apache.Ignite.Core.Impl.Common;
     using Apache.Ignite.Core.Impl.Compute;
     using Apache.Ignite.Core.Impl.Events;
@@ -43,7 +42,7 @@ namespace Apache.Ignite.Core.Impl.Cluster
     /// <summary>
     /// Ignite projection implementation.
     /// </summary>
-    internal class ClusterGroupImpl : PlatformTarget, IClusterGroupEx
+    internal class ClusterGroupImpl : PlatformTarget, IClusterGroup
     {
         /** Attribute: platform. */
         private const string AttrPlatform = "org.apache.ignite.platform";
@@ -55,9 +54,6 @@ namespace Apache.Ignite.Core.Impl.Cluster
         private const int TopVerInit = 0;
 
         /** */
-        private const int OpAllMetadata = 1;
-
-        /** */
         private const int OpForAttribute = 2;
 
         /** */
@@ -76,9 +72,6 @@ namespace Apache.Ignite.Core.Impl.Cluster
         private const int OpForNodeIds = 7;
 
         /** */
-        private const int OpMetadata = 8;
-
-        /** */
         private const int OpMetrics = 9;
 
         /** */
@@ -97,13 +90,10 @@ namespace Apache.Ignite.Core.Impl.Cluster
         private const int OpTopology = 14;
 
         /** */
-        private const int OpSchema = 15;
-
-        /** */
         private const int OpForRemotes = 17;
 
         /** */
-        public const int OpForDaemons = 18;
+        private const int OpForDaemons = 18;
 
         /** */
         private const int OpForRandom = 19;
@@ -115,10 +105,10 @@ namespace Apache.Ignite.Core.Impl.Cluster
         private const int OpForYoungest = 21;
         
         /** */
-        public const int OpResetMetrics = 22;
+        private const int OpResetMetrics = 22;
         
         /** */
-        public const int OpForServers = 23;
+        private const int OpForServers = 23;
         
         /** Initial Ignite instance. */
         private readonly Ignite _ignite;
@@ -508,6 +498,14 @@ namespace Apache.Ignite.Core.Impl.Cluster
         }
 
         /// <summary>
+        /// Resets the metrics.
+        /// </summary>
+        public void ResetMetrics()
+        {
+            DoOutInOp(OpResetMetrics);
+        }
+
+        /// <summary>
         /// Creates new Cluster Group from given native projection.
         /// </summary>
         /// <param name="prj">Native projection.</param>
@@ -553,51 +551,5 @@ namespace Apache.Ignite.Core.Impl.Cluster
 
             return _nodes;
         }
-
-        /** <inheritDoc /> */
-        public IBinaryType GetBinaryType(int typeId)
-        {
-            return DoOutInOp<IBinaryType>(OpMetadata,
-                writer => writer.WriteInt(typeId),
-                stream =>
-                {
-                    var reader = Marshaller.StartUnmarshal(stream, false);
-
-                    return reader.ReadBoolean() ? new BinaryType(reader) : null;
-                }
-            );
-        }
-
-        /// <summary>
-        /// Gets metadata for all known types.
-        /// </summary>
-        public List<IBinaryType> GetBinaryTypes()
-        {
-            return DoInOp(OpAllMetadata, s =>
-            {
-                var reader = Marshaller.StartUnmarshal(s);
-
-                var size = reader.ReadInt();
-
-                var res = new List<IBinaryType>(size);
-
-                for (var i = 0; i < size; i++)
-                    res.Add(reader.ReadBoolean() ? new BinaryType(reader) : null);
-
-                return res;
-            });
-        }
-
-        /// <summary>
-        /// Gets the schema.
-        /// </summary>
-        public int[] GetSchema(int typeId, int schemaId)
-        {
-            return DoOutInOp<int[]>(OpSchema, writer =>
-            {
-                writer.WriteInt(typeId);
-                writer.WriteInt(schemaId);
-            });
-        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/IClusterGroupEx.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/IClusterGroupEx.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/IClusterGroupEx.cs
deleted file mode 100644
index 59162e6..0000000
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/IClusterGroupEx.cs
+++ /dev/null
@@ -1,35 +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.
- */
-
-namespace Apache.Ignite.Core.Impl.Cluster
-{
-    using Apache.Ignite.Core.Binary;
-    using Apache.Ignite.Core.Cluster;
-
-    /// <summary>
-    /// Extended internal Ignite interface.
-    /// </summary>
-    internal interface IClusterGroupEx : IClusterGroup
-    {
-        /// <summary>
-        /// Gets protable metadata for type.
-        /// </summary>
-        /// <param name="typeId">Type ID.</param>
-        /// <returns>Metadata.</returns>
-        IBinaryType GetBinaryType(int typeId);
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs
index e8171cb..1747df2 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs
@@ -33,7 +33,6 @@ namespace Apache.Ignite.Core.Impl
     using Apache.Ignite.Core.DataStructures;
     using Apache.Ignite.Core.Events;
     using Apache.Ignite.Core.Impl.Binary;
-    using Apache.Ignite.Core.Impl.Binary.Metadata;
     using Apache.Ignite.Core.Impl.Cache;
     using Apache.Ignite.Core.Impl.Cluster;
     using Apache.Ignite.Core.Impl.Common;
@@ -52,7 +51,7 @@ namespace Apache.Ignite.Core.Impl
     /// <summary>
     /// Native Ignite wrapper.
     /// </summary>
-    internal class Ignite : IIgnite, IClusterGroupEx, ICluster
+    internal class Ignite : IIgnite, ICluster
     {
         /** */
         private readonly IgniteConfiguration _cfg;
@@ -72,6 +71,9 @@ namespace Apache.Ignite.Core.Impl
         /** Binary. */
         private readonly Binary.Binary _binary;
 
+        /** Binary processor. */
+        private readonly BinaryProcessor _binaryProc;
+
         /** Cached proxy. */
         private readonly IgniteProxy _proxy;
 
@@ -126,6 +128,8 @@ namespace Apache.Ignite.Core.Impl
 
             _binary = new Binary.Binary(marsh);
 
+            _binaryProc = new BinaryProcessor(UU.ProcessorBinaryProcessor(proc), marsh);
+
             _proxy = new IgniteProxy(this);
 
             cbs.Initialize(this);
@@ -517,7 +521,7 @@ namespace Apache.Ignite.Core.Impl
         /** <inheritdoc /> */
         public void ResetMetrics()
         {
-            UU.TargetInLongOutLong(_prj.Target, ClusterGroupImpl.OpResetMetrics, 0);
+            _prj.ResetMetrics();
         }
 
         /** <inheritdoc /> */
@@ -732,26 +736,19 @@ namespace Apache.Ignite.Core.Impl
         }
 
         /// <summary>
-        /// Configuration.
+        /// Gets the binary processor.
         /// </summary>
-        internal IgniteConfiguration Configuration
+        internal BinaryProcessor BinaryProcessor
         {
-            get { return _cfg; }
+            get { return _binaryProc; }
         }
 
         /// <summary>
-        /// Put metadata to Grid.
+        /// Configuration.
         /// </summary>
-        /// <param name="metas">Metadata.</param>
-        internal void PutBinaryTypes(ICollection<BinaryType> metas)
-        {
-            _prj.PutBinaryTypes(metas);
-        }
-
-        /** <inheritDoc /> */
-        public IBinaryType GetBinaryType(int typeId)
+        internal IgniteConfiguration Configuration
         {
-            return _prj.GetBinaryType(typeId);
+            get { return _cfg; }
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteProxy.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteProxy.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteProxy.cs
index 8dc63bd..98a2d47 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteProxy.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteProxy.cs
@@ -30,7 +30,6 @@ namespace Apache.Ignite.Core.Impl
     using Apache.Ignite.Core.DataStructures;
     using Apache.Ignite.Core.Events;
     using Apache.Ignite.Core.Impl.Binary;
-    using Apache.Ignite.Core.Impl.Cluster;
     using Apache.Ignite.Core.Log;
     using Apache.Ignite.Core.Lifecycle;
     using Apache.Ignite.Core.Messaging;
@@ -42,7 +41,7 @@ namespace Apache.Ignite.Core.Impl
     /// </summary>
     [Serializable]
     [ExcludeFromCodeCoverage]
-    internal class IgniteProxy : IIgnite, IClusterGroupEx, IBinaryWriteAware, ICluster
+    internal class IgniteProxy : IIgnite, IBinaryWriteAware, ICluster
     {
         /** */
         [NonSerialized]
@@ -455,11 +454,5 @@ namespace Apache.Ignite.Core.Impl
                 return _ignite;
             }
         }
-
-        /** <inheritdoc /> */
-        public IBinaryType GetBinaryType(int typeId)
-        {
-            return _ignite.GetBinaryType(typeId);
-        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs
index fb9d890..bafc759 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs
@@ -26,7 +26,6 @@ namespace Apache.Ignite.Core.Impl
     using Apache.Ignite.Core.Binary;
     using Apache.Ignite.Core.Impl.Binary;
     using Apache.Ignite.Core.Impl.Binary.IO;
-    using Apache.Ignite.Core.Impl.Binary.Metadata;
     using Apache.Ignite.Core.Impl.Common;
     using Apache.Ignite.Core.Impl.Memory;
     using Apache.Ignite.Core.Impl.Unmanaged;
@@ -50,9 +49,6 @@ namespace Apache.Ignite.Core.Impl
         protected const int Error = -1;
 
         /** */
-        private const int OpMeta = -1;
-
-        /** */
         public const int OpNone = -2;
 
         /** */
@@ -827,66 +823,6 @@ namespace Apache.Ignite.Core.Impl
         }
 
         /// <summary>
-        /// Put binary types to Grid.
-        /// </summary>
-        /// <param name="types">Binary types.</param>
-        internal void PutBinaryTypes(ICollection<BinaryType> types)
-        {
-            DoOutOp(OpMeta, stream =>
-            {
-                BinaryWriter w = _marsh.StartMarshal(stream);
-
-                w.WriteInt(types.Count);
-
-                foreach (var meta in types)
-                {
-                    w.WriteInt(meta.TypeId);
-                    w.WriteString(meta.TypeName);
-                    w.WriteString(meta.AffinityKeyFieldName);
-
-                    IDictionary<string, int> fields = meta.GetFieldsMap();
-
-                    w.WriteInt(fields.Count);
-
-                    foreach (var field in fields)
-                    {
-                        w.WriteString(field.Key);
-                        w.WriteInt(field.Value);
-                    }
-
-                    w.WriteBoolean(meta.IsEnum);
-
-                    // Send schemas
-                    var desc = meta.Descriptor;
-                    Debug.Assert(desc != null);
-
-                    var count = 0;
-                    var countPos = stream.Position;
-                    w.WriteInt(0);  // Reserve for count
-
-                    foreach (var schema in desc.Schema.GetAll())
-                    {
-                        w.WriteInt(schema.Key);
-
-                        var ids = schema.Value;
-                        w.WriteInt(ids.Length);
-
-                        foreach (var id in ids)
-                            w.WriteInt(id);
-
-                        count++;
-                    }
-
-                    stream.WriteInt(countPos, count);
-                }
-
-                _marsh.FinishMarshal(w);
-            });
-
-            _marsh.OnBinaryTypesSent(types);
-        }
-
-        /// <summary>
         /// Unmarshal object using the given stream.
         /// </summary>
         /// <param name="stream">Stream.</param>

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IgniteJniNativeMethods.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IgniteJniNativeMethods.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IgniteJniNativeMethods.cs
index ac41f25..c746866 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IgniteJniNativeMethods.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IgniteJniNativeMethods.cs
@@ -123,6 +123,9 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
         [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteProcessorLoggerLog")]
         public static extern void ProcessorLoggerLog(void* ctx, void* obj, int level, sbyte* messsage, sbyte* category, sbyte* errorInfo);
 
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteProcessorBinaryProcessor")]
+        public static extern void* ProcessorBinaryProcessor(void* ctx, void* obj);
+
         [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteTargetInStreamOutLong")]
         public static extern long TargetInStreamOutLong(void* ctx, void* target, int opType, long memPtr);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs
index fe1904c..f36c35f 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs
@@ -402,6 +402,13 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
             }
         }
 
+        internal static IUnmanagedTarget ProcessorBinaryProcessor(IUnmanagedTarget target)
+        {
+            void* res = JNI.ProcessorBinaryProcessor(target.Context, target.Target);
+
+            return target.ChangeTarget(res);
+        }
+
         #endregion
 
         #region NATIVE METHODS: TARGET