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/03/30 11:50:05 UTC

[36/50] [abbrv] ignite git commit: IGNITE-4617: CPP: Added Field-access methods for BinaryObject

IGNITE-4617: CPP: Added Field-access methods for BinaryObject


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

Branch: refs/heads/ignite-3477-master
Commit: d44de994e33973a7423f24df52ef50e2af9a1355
Parents: 8ae3d5b
Author: Igor Sapego <is...@gridgain.com>
Authored: Tue Mar 28 18:03:58 2017 +0300
Committer: Igor Sapego <is...@gridgain.com>
Committed: Tue Mar 28 18:03:58 2017 +0300

----------------------------------------------------------------------
 .../internal/binary/BinaryClassDescriptor.java  |  10 +-
 .../ignite/internal/binary/BinaryContext.java   |   2 +-
 .../internal/binary/BinaryFieldMetadata.java    | 127 +++++++++++
 .../ignite/internal/binary/BinaryMetadata.java  |  20 +-
 .../binary/BinaryMetadataCollector.java         |  17 +-
 .../ignite/internal/binary/BinaryUtils.java     |  14 +-
 .../binary/builder/BinaryObjectBuilderImpl.java |  14 +-
 .../binary/CacheObjectBinaryProcessor.java      |   3 +-
 .../binary/CacheObjectBinaryProcessorImpl.java  |   3 +-
 .../platform/PlatformContextImpl.java           |  82 +++----
 modules/platforms/cpp/binary/Makefile.am        |   2 +-
 .../platforms/cpp/binary/include/Makefile.am    |   1 +
 .../include/ignite/binary/binary_object.h       |  75 ++++++-
 .../ignite/impl/binary/binary_field_meta.h      | 110 ++++++++++
 .../ignite/impl/binary/binary_id_resolver.h     |  96 +++++++-
 .../ignite/impl/binary/binary_object_header.h   |  13 ++
 .../ignite/impl/binary/binary_object_impl.h     | 124 +++++++++--
 .../include/ignite/impl/binary/binary_schema.h  |  10 +-
 .../ignite/impl/binary/binary_type_handler.h    |  47 ++--
 .../ignite/impl/binary/binary_type_manager.h    |  48 ++--
 .../ignite/impl/binary/binary_type_snapshot.h   |  82 +++++--
 .../ignite/impl/binary/binary_type_updater.h    |  19 +-
 .../ignite/impl/binary/binary_writer_impl.h     |  17 +-
 .../cpp/binary/project/vs/binary.vcxproj        |   3 +-
 .../binary/project/vs/binary.vcxproj.filters    |   9 +-
 .../src/impl/binary/binary_field_meta.cpp       |  42 ++++
 .../src/impl/binary/binary_object_impl.cpp      | 139 +++++++++++-
 .../src/impl/binary/binary_type_handler.cpp     |  45 +---
 .../src/impl/binary/binary_type_manager.cpp     | 187 +++++++---------
 .../src/impl/binary/binary_type_snapshot.cpp    |  50 ++---
 .../src/impl/binary/binary_type_updater.cpp     |  32 ---
 .../cpp/core-test/config/cache-identity.xml     |  33 +++
 .../core-test/include/ignite/binary_test_defs.h |   5 +
 .../src/binary_identity_resolver_test.cpp       |  91 +++++++-
 .../cpp/core-test/src/binary_object_test.cpp    | 220 ++++++++++++++++++-
 .../cpp/core-test/src/cache_invoke_test.cpp     |   2 +
 .../cpp/core-test/src/cluster_test.cpp          |   5 +-
 .../impl/binary/binary_type_updater_impl.h      |   6 +-
 .../cpp/core/include/ignite/impl/ignite_impl.h  |  11 +
 .../ignite/impl/interop/interop_target.h        |   2 +-
 .../impl/binary/binary_type_updater_impl.cpp    | 101 +++++++--
 .../cpp/core/src/impl/ignite_environment.cpp    |   4 +-
 .../core/src/impl/interop/interop_target.cpp    |   2 +-
 .../Apache.Ignite.Core.csproj                   |   1 +
 .../Impl/Binary/BinaryObjectBuilder.cs          |   2 +-
 .../Impl/Binary/BinaryProcessor.cs              |   3 +-
 .../Impl/Binary/BinaryWriter.cs                 |   2 +-
 .../Impl/Binary/Marshaller.cs                   |   6 +-
 .../Impl/Binary/Metadata/BinaryField.cs         |  72 ++++++
 .../Impl/Binary/Metadata/BinaryType.cs          |  38 +++-
 .../Binary/Metadata/BinaryTypeHashsetHandler.cs |  10 +-
 .../Impl/Binary/Metadata/BinaryTypeHolder.cs    |   9 +-
 .../Impl/Binary/Metadata/IBinaryTypeHandler.cs  |   2 +-
 53 files changed, 1586 insertions(+), 484 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d44de994/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java
index 7eaf6c6..92e4874 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java
@@ -101,7 +101,7 @@ public class BinaryClassDescriptor {
     private final Method readResolveMtd;
 
     /** */
-    private final Map<String, Integer> stableFieldsMeta;
+    private final Map<String, BinaryFieldMetadata> stableFieldsMeta;
 
     /** Object schemas. Initialized only for serializable classes and contains only 1 entry. */
     private final BinarySchema stableSchema;
@@ -279,12 +279,12 @@ public class BinaryClassDescriptor {
                 if (BinaryUtils.FIELDS_SORTED_ORDER) {
                     fields0 = new TreeMap<>();
 
-                    stableFieldsMeta = metaDataEnabled ? new TreeMap<String, Integer>() : null;
+                    stableFieldsMeta = metaDataEnabled ? new TreeMap<String, BinaryFieldMetadata>() : null;
                 }
                 else {
                     fields0 = new LinkedHashMap<>();
 
-                    stableFieldsMeta = metaDataEnabled ? new LinkedHashMap<String, Integer>() : null;
+                    stableFieldsMeta = metaDataEnabled ? new LinkedHashMap<String, BinaryFieldMetadata>() : null;
                 }
 
                 Set<String> duplicates = duplicateFields(cls);
@@ -316,7 +316,7 @@ public class BinaryClassDescriptor {
                             fields0.put(name, fieldInfo);
 
                             if (metaDataEnabled)
-                                stableFieldsMeta.put(name, fieldInfo.mode().typeId());
+                                stableFieldsMeta.put(name, new BinaryFieldMetadata(fieldInfo));
                         }
                     }
                 }
@@ -462,7 +462,7 @@ public class BinaryClassDescriptor {
     /**
      * @return Fields meta data.
      */
-    Map<String, Integer> fieldsMeta() {
+    Map<String, BinaryFieldMetadata> fieldsMeta() {
         return stableFieldsMeta;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/d44de994/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
index 5f9e4ba..e5b6bda 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
@@ -1140,7 +1140,7 @@ public class BinaryContext {
 
         cls2Mappers.put(clsName, mapper);
 
-        Map<String, Integer> fieldsMeta = null;
+        Map<String, BinaryFieldMetadata> fieldsMeta = null;
 
         if (cls != null) {
             if (serializer == null) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/d44de994/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryFieldMetadata.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryFieldMetadata.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryFieldMetadata.java
new file mode 100644
index 0000000..e1aef86
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryFieldMetadata.java
@@ -0,0 +1,127 @@
+/*
+ * 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.binary;
+
+import org.apache.ignite.internal.util.tostring.GridToStringInclude;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+
+/**
+ * Binary field metadata.
+ */
+public class BinaryFieldMetadata implements Externalizable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Field id in schema. */
+    @GridToStringInclude(sensitive = true)
+    private int fieldId;
+
+    /** Field type ID. */
+    @GridToStringInclude(sensitive = true)
+    private int typeId;
+
+    /**
+     * For {@link Externalizable}.
+     */
+    public BinaryFieldMetadata() {
+        // No-op.
+    }
+
+    /**
+     * Constructor.
+     * @param typeId Field type ID.
+     * @param fieldId Field id in schema.
+     */
+    public BinaryFieldMetadata(int typeId, int fieldId) {
+        this.typeId = typeId;
+        this.fieldId = fieldId;
+    }
+
+    /**
+     * Constructor.
+     * @param accessor Field accessor.
+     */
+    public BinaryFieldMetadata(BinaryFieldAccessor accessor) {
+        this.typeId = accessor.mode().typeId();
+        this.fieldId = accessor.id;
+    }
+
+    /**
+     * @return Field ID in binary schema.
+     */
+    public int fieldId() {
+        return fieldId;
+    }
+
+    /**
+     * @return ID of the type of the field.
+     */
+    public int typeId() {
+        return typeId;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        writeTo(out);
+    }
+
+    /**
+     * The object implements the writeTo method to save its contents
+     * by calling the methods of DataOutput for its primitive values and strings or
+     * calling the writeTo method for other objects.
+     *
+     * @param out the stream to write the object to.
+     * @exception IOException Includes any I/O exceptions that may occur.
+     */
+    public void writeTo(DataOutput out) throws IOException {
+        out.writeInt(typeId);
+        out.writeInt(fieldId);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        readFrom(in);
+    }
+
+    /**
+     * The object implements the readFrom method to restore its
+     * contents by calling the methods of DataInput for primitive
+     * types and strings or calling readExternal for other objects.  The
+     * readFrom method must read the values in the same sequence
+     * and with the same types as were written by writeTo.
+     *
+     * @param in the stream to read data from in order to restore the object.
+     * @exception IOException if I/O errors occur.
+     */
+    public void readFrom(DataInput in) throws IOException {
+        typeId = in.readInt();
+        fieldId = in.readInt();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(BinaryFieldMetadata.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/d44de994/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMetadata.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMetadata.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMetadata.java
index ec92b08..a2589bf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMetadata.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMetadata.java
@@ -51,7 +51,7 @@ public class BinaryMetadata implements Externalizable {
 
     /** Recorded object fields. */
     @GridToStringInclude(sensitive = true)
-    private Map<String, Integer> fields;
+    private Map<String, BinaryFieldMetadata> fields;
 
     /** Affinity key field name. */
     @GridToStringInclude(sensitive = true)
@@ -80,7 +80,7 @@ public class BinaryMetadata implements Externalizable {
      * @param schemas Schemas.
      * @param isEnum Enum flag.
      */
-    public BinaryMetadata(int typeId, String typeName, @Nullable Map<String, Integer> fields,
+    public BinaryMetadata(int typeId, String typeName, @Nullable Map<String, BinaryFieldMetadata> fields,
         @Nullable String affKeyFieldName, @Nullable Collection<BinarySchema> schemas, boolean isEnum) {
         assert typeName != null;
 
@@ -116,8 +116,8 @@ public class BinaryMetadata implements Externalizable {
     /**
      * @return Fields.
      */
-    public Map<String, Integer> fieldsMap() {
-        return fields != null ? fields : Collections.<String, Integer>emptyMap();
+    public Map<String, BinaryFieldMetadata> fieldsMap() {
+        return fields != null ? fields : Collections.<String, BinaryFieldMetadata>emptyMap();
     }
 
     /**
@@ -125,7 +125,7 @@ public class BinaryMetadata implements Externalizable {
      * @return Field type name.
      */
     @Nullable public String fieldTypeName(String fieldName) {
-        Integer typeId = fields != null ? fields.get(fieldName) : null;
+        Integer typeId = fields != null ? fields.get(fieldName).typeId() : null;
 
         return typeId != null ? BinaryUtils.fieldTypeName(typeId) : null;
     }
@@ -184,9 +184,9 @@ public class BinaryMetadata implements Externalizable {
         else {
             out.writeInt(fields.size());
 
-            for (Map.Entry<String, Integer> fieldEntry : fields.entrySet()) {
+            for (Map.Entry<String, BinaryFieldMetadata> fieldEntry : fields.entrySet()) {
                 U.writeString(out, fieldEntry.getKey());
-                out.writeInt(fieldEntry.getValue());
+                fieldEntry.getValue().writeTo(out);
             }
         }
 
@@ -232,9 +232,11 @@ public class BinaryMetadata implements Externalizable {
 
             for (int i = 0; i < fieldsSize; i++) {
                 String fieldName = U.readString(in);
-                int fieldId = in.readInt();
 
-                fields.put(fieldName, fieldId);
+                BinaryFieldMetadata fieldMeta = new BinaryFieldMetadata();
+                fieldMeta.readFrom(in);
+
+                fields.put(fieldName, fieldMeta);
             }
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/d44de994/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMetadataCollector.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMetadataCollector.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMetadataCollector.java
index 2585b3b..22d2001 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMetadataCollector.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMetadataCollector.java
@@ -47,7 +47,7 @@ class BinaryMetadataCollector implements BinaryWriter {
     private final BinaryInternalMapper mapper;
 
     /** Collected metadata. */
-    private final Map<String, Integer> meta = new HashMap<>();
+    private final Map<String, BinaryFieldMetadata> meta = new HashMap<>();
 
     /** Schema builder. */
     private BinarySchema.Builder schemaBuilder = BinarySchema.Builder.newBuilder();
@@ -68,7 +68,7 @@ class BinaryMetadataCollector implements BinaryWriter {
     /**
      * @return Field meta data.
      */
-    Map<String, Integer> meta() {
+    Map<String, BinaryFieldMetadata> meta() {
         return meta;
     }
 
@@ -269,18 +269,19 @@ class BinaryMetadataCollector implements BinaryWriter {
     private void add(String name, BinaryWriteMode mode) throws BinaryObjectException {
         assert name != null;
 
-        int fieldTypeId = mode.typeId();
+        int typeId = mode.typeId();
+        int fieldId = mapper.fieldId(typeId, name);
 
-        Integer oldFieldTypeId = meta.put(name, fieldTypeId);
+        BinaryFieldMetadata oldFieldMeta = meta.put(name, new BinaryFieldMetadata(typeId, fieldId));
 
-        if (oldFieldTypeId != null && !oldFieldTypeId.equals(fieldTypeId)) {
+        if (oldFieldMeta != null && oldFieldMeta.typeId() != typeId) {
             throw new BinaryObjectException(
                 "Field is written twice with different types [" + "typeName=" + typeName + ", fieldName=" + name +
-                ", fieldTypeName1=" + BinaryUtils.fieldTypeName(oldFieldTypeId) +
-                ", fieldTypeName2=" + BinaryUtils.fieldTypeName(fieldTypeId) + ']'
+                ", fieldTypeName1=" + BinaryUtils.fieldTypeName(oldFieldMeta.typeId()) +
+                ", fieldTypeName2=" + BinaryUtils.fieldTypeName(typeId) + ']'
             );
         }
 
-        schemaBuilder.addField(mapper.fieldId(typeId, name));
+        schemaBuilder.addField(fieldId);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/d44de994/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java
index d3ff2ac..41ec078 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java
@@ -960,25 +960,25 @@ public class BinaryUtils {
             }
 
             // Check and merge fields.
-            Map<String, Integer> mergedFields;
+            Map<String, BinaryFieldMetadata> mergedFields;
 
             if (FIELDS_SORTED_ORDER)
                 mergedFields = new TreeMap<>(oldMeta.fieldsMap());
             else
                 mergedFields = new LinkedHashMap<>(oldMeta.fieldsMap());
 
-            Map<String, Integer> newFields = newMeta.fieldsMap();
+            Map<String, BinaryFieldMetadata> newFields = newMeta.fieldsMap();
 
             boolean changed = false;
 
-            for (Map.Entry<String, Integer> newField : newFields.entrySet()) {
-                Integer oldFieldType = mergedFields.put(newField.getKey(), newField.getValue());
+            for (Map.Entry<String, BinaryFieldMetadata> newField : newFields.entrySet()) {
+                BinaryFieldMetadata oldFieldMeta = mergedFields.put(newField.getKey(), newField.getValue());
 
-                if (oldFieldType == null)
+                if (oldFieldMeta == null)
                     changed = true;
                 else {
-                    String oldFieldTypeName = fieldTypeName(oldFieldType);
-                    String newFieldTypeName = fieldTypeName(newField.getValue());
+                    String oldFieldTypeName = fieldTypeName(oldFieldMeta.typeId());
+                    String newFieldTypeName = fieldTypeName(newField.getValue().typeId());
 
                     if (!F.eq(oldFieldTypeName, newFieldTypeName)) {
                         throw new BinaryObjectException(

http://git-wip-us.apache.org/repos/asf/ignite/blob/d44de994/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java
index 4ab9df8..6827c96 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java
@@ -28,6 +28,7 @@ import org.apache.ignite.internal.binary.BinaryObjectImpl;
 import org.apache.ignite.internal.binary.BinaryWriterExImpl;
 import org.apache.ignite.internal.binary.GridBinaryMarshaller;
 import org.apache.ignite.internal.binary.BinaryContext;
+import org.apache.ignite.internal.binary.BinaryFieldMetadata;
 import org.apache.ignite.internal.binary.BinarySchema;
 import org.apache.ignite.internal.binary.BinarySchemaRegistry;
 import org.apache.ignite.internal.binary.BinaryObjectOffheapImpl;
@@ -197,7 +198,7 @@ public class BinaryObjectBuilderImpl implements BinaryObjectBuilder {
 
             BinaryType meta = ctx.metadata(typeId);
 
-            Map<String, Integer> fieldsMeta = null;
+            Map<String, BinaryFieldMetadata> fieldsMeta = null;
 
             if (reader != null && BinaryUtils.hasSchema(flags)) {
                 BinarySchema schema = reader.schema();
@@ -216,7 +217,7 @@ public class BinaryObjectBuilderImpl implements BinaryObjectBuilder {
                         assignedFldsById.put(fieldId, val);
 
                         if (val != REMOVED_FIELD_MARKER)
-                            fieldsMeta = checkMetadata(meta, fieldsMeta, val, name);
+                            fieldsMeta = checkMetadata(meta, fieldsMeta, val, name, fieldId);
                     }
 
                     remainsFlds = assignedFldsById.keySet();
@@ -310,7 +311,7 @@ public class BinaryObjectBuilderImpl implements BinaryObjectBuilder {
 
                     if (reader == null)
                         // Metadata has already been checked.
-                        fieldsMeta = checkMetadata(meta, fieldsMeta, val, name);
+                        fieldsMeta = checkMetadata(meta, fieldsMeta, val, name, fieldId);
                 }
             }
 
@@ -369,9 +370,10 @@ public class BinaryObjectBuilderImpl implements BinaryObjectBuilder {
      * @param fieldsMeta Map holding metadata information that has to be updated.
      * @param newVal Field value being serialized.
      * @param name Field name.
+     * @param fieldId Field ID.
      */
-    private Map<String, Integer> checkMetadata(BinaryType meta, Map<String, Integer> fieldsMeta, Object newVal,
-        String name) {
+    private Map<String, BinaryFieldMetadata> checkMetadata(BinaryType meta, Map<String, BinaryFieldMetadata> fieldsMeta,
+        Object newVal, String name, int fieldId) {
         String oldFldTypeName = meta == null ? null : meta.fieldTypeName(name);
 
         boolean nullFieldVal = false;
@@ -405,7 +407,7 @@ public class BinaryObjectBuilderImpl implements BinaryObjectBuilder {
                     fieldsMeta = new LinkedHashMap<>();
             }
 
-            fieldsMeta.put(name, newFldTypeId);
+            fieldsMeta.put(name, new BinaryFieldMetadata(newFldTypeId, fieldId));
         }
         else if (!nullFieldVal) {
             String newFldTypeName = BinaryUtils.fieldTypeName(newFldTypeId);

http://git-wip-us.apache.org/repos/asf/ignite/blob/d44de994/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessor.java
index 3578f7a..7cc7a29 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessor.java
@@ -21,6 +21,7 @@ import java.util.Collection;
 import java.util.Map;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteBinary;
+import org.apache.ignite.internal.binary.BinaryFieldMetadata;
 import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor;
 import org.apache.ignite.binary.BinaryObjectBuilder;
 import org.apache.ignite.binary.BinaryType;
@@ -61,7 +62,7 @@ public interface CacheObjectBinaryProcessor extends IgniteCacheObjectProcessor {
      * @throws IgniteException In case of error.
      */
     public void updateMetadata(int typeId, String typeName, @Nullable String affKeyFieldName,
-        Map<String, Integer> fieldTypeIds, boolean isEnum) throws IgniteException;
+        Map<String, BinaryFieldMetadata> fieldTypeIds, boolean isEnum) throws IgniteException;
 
     /**
      * @param typeId Type ID.

http://git-wip-us.apache.org/repos/asf/ignite/blob/d44de994/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
index 656e70a..6e5940f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
@@ -55,6 +55,7 @@ import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.IgniteNodeAttributes;
 import org.apache.ignite.internal.binary.BinaryContext;
 import org.apache.ignite.internal.binary.BinaryEnumObjectImpl;
+import org.apache.ignite.internal.binary.BinaryFieldMetadata;
 import org.apache.ignite.internal.binary.BinaryMarshaller;
 import org.apache.ignite.internal.binary.BinaryMetadata;
 import org.apache.ignite.internal.binary.BinaryMetadataHandler;
@@ -522,7 +523,7 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
 
     /** {@inheritDoc} */
     @Override public void updateMetadata(int typeId, String typeName, @Nullable String affKeyFieldName,
-        Map<String, Integer> fieldTypeIds, boolean isEnum) throws BinaryObjectException {
+        Map<String, BinaryFieldMetadata> fieldTypeIds, boolean isEnum) throws BinaryObjectException {
         BinaryMetadata meta = new BinaryMetadata(typeId, typeName, fieldTypeIds, affKeyFieldName, null, isEnum);
 
         binaryCtx.updateMetadata(typeId, meta);

http://git-wip-us.apache.org/repos/asf/ignite/blob/d44de994/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java
index 8f7d5de..10a8f74 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java
@@ -35,6 +35,7 @@ import org.apache.ignite.events.SwapSpaceEvent;
 import org.apache.ignite.events.TaskEvent;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.binary.BinaryContext;
+import org.apache.ignite.internal.binary.BinaryFieldMetadata;
 import org.apache.ignite.internal.binary.BinaryMetadata;
 import org.apache.ignite.internal.binary.BinaryRawReaderEx;
 import org.apache.ignite.internal.binary.BinaryRawWriterEx;
@@ -354,17 +355,22 @@ public class PlatformContextImpl implements PlatformContext {
     /** {@inheritDoc} */
     @SuppressWarnings("ConstantConditions")
     @Override public void processMetadata(BinaryRawReaderEx reader) {
-        Collection<Metadata> metas = PlatformUtils.readCollection(reader,
-            new PlatformReaderClosure<Metadata>() {
-                @Override public Metadata read(BinaryRawReaderEx reader) {
+        Collection<BinaryMetadata> metas = PlatformUtils.readCollection(reader,
+            new PlatformReaderClosure<BinaryMetadata>() {
+                @Override public BinaryMetadata read(BinaryRawReaderEx reader) {
                     int typeId = reader.readInt();
                     String typeName = reader.readString();
                     String affKey = reader.readString();
 
-                    Map<String, Integer> fields = PlatformUtils.readLinkedMap(reader,
-                        new PlatformReaderBiClosure<String, Integer>() {
-                            @Override public IgniteBiTuple<String, Integer> read(BinaryRawReaderEx reader) {
-                                return F.t(reader.readString(), reader.readInt());
+                    Map<String, BinaryFieldMetadata> fields = PlatformUtils.readLinkedMap(reader,
+                        new PlatformReaderBiClosure<String, BinaryFieldMetadata>() {
+                            @Override public IgniteBiTuple<String, BinaryFieldMetadata> read(BinaryRawReaderEx reader) {
+                                String name = reader.readString();
+                                int typeId = reader.readInt();
+                                int fieldId = reader.readInt();
+
+                                return new IgniteBiTuple<String, BinaryFieldMetadata>(name,
+                                        new BinaryFieldMetadata(typeId, fieldId));
                             }
                         });
 
@@ -390,16 +396,15 @@ public class PlatformContextImpl implements PlatformContext {
                         }
                     }
 
-                    return new Metadata(typeId, typeName, affKey, fields, isEnum, schemas);
+                    return new BinaryMetadata(typeId, typeName, fields, affKey, schemas, isEnum);
                 }
             }
         );
 
         BinaryContext binCtx = cacheObjProc.binaryContext();
 
-        for (Metadata meta : metas)
-            binCtx.updateMetadata(meta.typeId, new BinaryMetadata(meta.typeId,
-                meta.typeName, meta.fields, meta.affKey, meta.schemas, meta.isEnum));
+        for (BinaryMetadata meta : metas)
+            binCtx.updateMetadata(meta.typeId(), meta);
     }
 
     /** {@inheritDoc} */
@@ -455,12 +460,21 @@ public class PlatformContextImpl implements PlatformContext {
             writer.writeBoolean(true);
 
             BinaryMetadata meta0 = ((BinaryTypeImpl) meta).metadata();
-            Map<String, Integer> fields = meta0.fieldsMap();
+            Map<String, BinaryFieldMetadata> fields = meta0.fieldsMap();
 
             writer.writeInt(typeId);
             writer.writeString(meta.typeName());
             writer.writeString(meta.affinityKeyFieldName());
-            writer.writeMap(fields);
+
+            writer.writeInt(fields.size());
+
+            for (Map.Entry<String, BinaryFieldMetadata> e : fields.entrySet()) {
+                writer.writeString(e.getKey());
+
+                writer.writeInt(e.getValue().typeId());
+                writer.writeInt(e.getValue().fieldId());
+            }
+
             writer.writeBoolean(meta.isEnum());
         }
     }
@@ -683,46 +697,4 @@ public class PlatformContextImpl implements PlatformContext {
     @Override public String platform() {
         return platform;
     }
-
-    /**
-     * Metadata holder.
-     */
-    private static class Metadata {
-        /** Type ID. */
-        private final int typeId;
-
-        /** Type name. */
-        private final String typeName;
-
-        /** Affinity key. */
-        private final String affKey;
-
-        /** Fields map. */
-        private final Map<String, Integer> fields;
-
-        /** Enum flag. */
-        private final boolean isEnum;
-
-        /** Schemas. */
-        private final List<BinarySchema> schemas;
-
-        /**
-         * Constructor.
-         *  @param typeId Type ID.
-         * @param typeName Type name.
-         * @param affKey Affinity key.
-         * @param fields Fields.
-         * @param isEnum Enum flag.
-         * @param schemas Schemas.
-         */
-        private Metadata(int typeId, String typeName, String affKey, Map<String, Integer> fields, boolean isEnum,
-            List<BinarySchema> schemas) {
-            this.typeId = typeId;
-            this.typeName = typeName;
-            this.affKey = affKey;
-            this.fields = fields;
-            this.isEnum = isEnum;
-            this.schemas = schemas;
-        }
-    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/d44de994/modules/platforms/cpp/binary/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/binary/Makefile.am b/modules/platforms/cpp/binary/Makefile.am
index c13a1bb..d310f33 100644
--- a/modules/platforms/cpp/binary/Makefile.am
+++ b/modules/platforms/cpp/binary/Makefile.am
@@ -58,11 +58,11 @@ libignite_binary_la_SOURCES = \
     src/impl/binary/binary_reader_impl.cpp \
     src/impl/binary/binary_type_handler.cpp \
     src/impl/binary/binary_writer_impl.cpp \
-    src/impl/binary/binary_type_updater.cpp \
     src/impl/binary/binary_schema.cpp \
     src/impl/binary/binary_type_snapshot.cpp \
     src/impl/binary/binary_object_header.cpp \
     src/impl/binary/binary_object_impl.cpp \
+    src/impl/binary/binary_field_meta.cpp \
     src/impl/interop/interop_memory.cpp \
     src/impl/interop/interop_output_stream.cpp \
     src/impl/interop/interop_input_stream.cpp

http://git-wip-us.apache.org/repos/asf/ignite/blob/d44de994/modules/platforms/cpp/binary/include/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/binary/include/Makefile.am b/modules/platforms/cpp/binary/include/Makefile.am
index a25a753..9c51326 100644
--- a/modules/platforms/cpp/binary/include/Makefile.am
+++ b/modules/platforms/cpp/binary/include/Makefile.am
@@ -35,6 +35,7 @@ nobase_include_HEADERS = \
     ignite/impl/binary/binary_type_updater.h \
     ignite/impl/binary/binary_common.h \
     ignite/impl/binary/binary_writer_impl.h \
+    ignite/impl/binary/binary_field_meta.h \
     ignite/impl/binary/binary_type_snapshot.h \
     ignite/impl/binary/binary_reader_impl.h \
     ignite/impl/binary/binary_schema.h \

http://git-wip-us.apache.org/repos/asf/ignite/blob/d44de994/modules/platforms/cpp/binary/include/ignite/binary/binary_object.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/binary/include/ignite/binary/binary_object.h b/modules/platforms/cpp/binary/include/ignite/binary/binary_object.h
index 12ac938..ff4bd10 100644
--- a/modules/platforms/cpp/binary/include/ignite/binary/binary_object.h
+++ b/modules/platforms/cpp/binary/include/ignite/binary/binary_object.h
@@ -53,24 +53,52 @@ namespace ignite
                 impl(impl)
             {
                 // No-op.
-            };
+            }
 
             /**
-             * Constructor.
+             * Direct constructor.
+             * Constructs binary object without any safety checks.
              *
              * @param mem Binary object memory.
              * @param start Object starting position in memory.
+             * @param idRslvr ID resolver.
              */
-            BinaryObject(impl::interop::InteropMemory& mem, int32_t start) :
-                impl(mem, start)
+            BinaryObject(impl::interop::InteropMemory& mem, int32_t start,
+                impl::binary::BinaryIdResolver* idRslvr, impl::binary::BinaryTypeManager* metaMgr) :
+                impl(mem, start, idRslvr, metaMgr)
             {
                 // No-op.
-            };
+            }
             /// @endcond
 
             /**
+             * Copy constructor.
+             *
+             * @param other Another instance.
+             */
+            BinaryObject(const BinaryObject& other) :
+                impl(other.impl)
+            {
+                // No-op.
+            }
+
+            /**
+             * Assignment operator.
+             *
+             * @param other Another instance.
+             * @return *this.
+             */
+            BinaryObject& operator=(const BinaryObject& other)
+            {
+                impl = other.impl;
+
+                return *this;
+            }
+
+            /**
              * Deserialize object.
-             * @throw IgniteError if the object can not be deserialized to specified type.
+             * @throw IgniteError if the object can not be deserialized to
+             *     specified type.
              *
              * @return Deserialized value.
              */
@@ -80,12 +108,43 @@ namespace ignite
                 return impl.Deserialize<T>();
             }
 
+            /**
+             * Get field.
+             * @throw IgniteError if the there is no specified field or if it
+             *     is not of the specified type.
+             *
+             * @param name Field name.
+             * @return Field value.
+             */
+            template<typename T>
+            T GetField(const char* name) const
+            {
+                return impl.GetField<T>(name);
+            }
+
+            /**
+             * Check if the binary object has the specified field.
+             *
+             * @param name Field name.
+             * @return True if the binary object has the specified field and
+             *     false otherwise.
+             */
+            bool HasField(const char* name) const
+            {
+                return impl.HasField(name);
+            }
+
         private:
             /** Implementation. */
             impl::binary::BinaryObjectImpl impl;
-
-            IGNITE_NO_COPY_ASSIGNMENT(BinaryObject)
         };
+
+        /* Specialization */
+        template<>
+        inline BinaryObject BinaryObject::GetField(const char* name) const
+        {
+            return BinaryObject(impl.GetField<impl::binary::BinaryObjectImpl>(name));
+        }
     }
 }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/d44de994/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_field_meta.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_field_meta.h b/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_field_meta.h
new file mode 100644
index 0000000..73864eb
--- /dev/null
+++ b/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_field_meta.h
@@ -0,0 +1,110 @@
+/*
+ * 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.
+ */
+
+#ifndef _IGNITE_IMPL_BINARY_BINARY_FIELD_META
+#define _IGNITE_IMPL_BINARY_BINARY_FIELD_META
+
+#include <stdint.h>
+
+namespace ignite
+{
+    namespace binary
+    {
+        /* Forward declarations. */
+        class BinaryRawWriter;
+        class BinaryRawReader;
+    }
+
+    namespace impl
+    {
+        namespace binary
+        {
+            /**
+             * Field metadata.
+             */
+            class BinaryFieldMeta
+            {
+            public:
+                /**
+                 * Default constructor.
+                 */
+                BinaryFieldMeta() :
+                    typeId(0),
+                    fieldId(0)
+                {
+                    // No-op.
+                }
+
+                /**
+                 * Constructor.
+                 *
+                 * @param typeId Type ID.
+                 * @param fieldId Field IDs.
+                 */
+                BinaryFieldMeta(int32_t typeId, int32_t fieldId) :
+                    typeId(typeId),
+                    fieldId(fieldId)
+                {
+                    // No-op.
+                }
+
+                /**
+                 * Get type ID.
+                 *
+                 * @return Type ID.
+                 */
+                int32_t GetTypeId() const
+                {
+                    return typeId;
+                }
+
+                /** 
+                 * Get field ID.
+                 *
+                 * @return Field ID.
+                 */
+                int32_t GetFieldId() const
+                {
+                    return fieldId;
+                }
+
+                /**
+                 * Write to data stream.
+                 *
+                 * @param writer Writer.
+                 */
+                void Write(ignite::binary::BinaryRawWriter& writer) const;
+
+                /**
+                 * Read from data stream.
+                 *
+                 * @param reader reader.
+                 */
+                void Read(ignite::binary::BinaryRawReader& reader);
+
+            private:
+                /** Type ID. */
+                int32_t typeId;
+
+                /** Field ID. */
+                int32_t fieldId;
+            };
+        }
+    }
+}
+
+#endif //_IGNITE_IMPL_BINARY_BINARY_FIELD_META
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/d44de994/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_id_resolver.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_id_resolver.h b/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_id_resolver.h
index 41ae628..7d6b12b 100644
--- a/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_id_resolver.h
+++ b/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_id_resolver.h
@@ -18,7 +18,11 @@
 #ifndef _IGNITE_IMPL_BINARY_BINARY_ID_RESOLVER
 #define _IGNITE_IMPL_BINARY_BINARY_ID_RESOLVER
 
+#include <map>
+
+#include "ignite/common/concurrent.h"
 #include "ignite/binary/binary_type.h"
+#include "ignite/impl/binary/binary_type_handler.h"
 
 namespace ignite
 {
@@ -55,10 +59,17 @@ namespace ignite
                  * @return Field ID.
                  */
                 virtual int32_t GetFieldId(const int32_t typeId, const char* name) = 0;
+
+                /**
+                 * Get copy of the instance.
+                 *
+                 * @return Copy of the instance.
+                 */
+                virtual BinaryIdResolver* Clone() const = 0;
             };
 
             /**
-             * Templated binary type descriptor.
+             * Templated binary type resolver.
              */
             template<typename T>
             class TemplatedBinaryIdResolver : public BinaryIdResolver
@@ -67,9 +78,10 @@ namespace ignite
                 /**
                  * Constructor.
                  */
-                TemplatedBinaryIdResolver()
+                TemplatedBinaryIdResolver() : 
+                    type()
                 {
-                    type = ignite::binary::BinaryType<T>();
+                    // No-op.
                 }
 
                 /**
@@ -77,7 +89,8 @@ namespace ignite
                  *
                  * @param type Binary type.
                  */
-                TemplatedBinaryIdResolver(ignite::binary::BinaryType<T> type) : type(type)
+                TemplatedBinaryIdResolver(ignite::binary::BinaryType<T> type) :
+                    type(type)
                 {
                     // No-op.
                 }
@@ -91,12 +104,85 @@ namespace ignite
                     if (name)
                         return type.GetFieldId(name);
                     else
-                        IGNITE_ERROR_1(IgniteError::IGNITE_ERR_BINARY, "Field name cannot be NULL.");
+                    {
+                        IGNITE_ERROR_FORMATTED_1(IgniteError::IGNITE_ERR_BINARY,
+                            "Field name cannot be NULL.", "typeId", typeId);
+                    }
                 }
+
+                virtual BinaryIdResolver* Clone() const
+                {
+                    return new TemplatedBinaryIdResolver<T>(*this);
+                }
+
             private:
                 /** Actual type.  */
                 ignite::binary::BinaryType<T> type; 
             };
+
+            /**
+             * Metadata binary type resolver.
+             */
+            class MetadataBinaryIdResolver : public BinaryIdResolver
+            {
+            public:
+                /**
+                 * Constructor.
+                 */
+                MetadataBinaryIdResolver() :
+                    meta()
+                {
+                    // No-op.
+                }
+
+                /**
+                 * Constructor.
+                 *
+                 * @param meta Binary type metadata snapshot.
+                 */
+                MetadataBinaryIdResolver(SPSnap meta) :
+                    meta(meta)
+                {
+                    // No-op.
+                }
+
+                virtual int32_t GetTypeId()
+                {
+                    return meta.Get()->GetTypeId();
+                }
+
+                virtual int32_t GetFieldId(const int32_t typeId, const char* name)
+                {
+                    if (!name)
+                    {
+                        IGNITE_ERROR_FORMATTED_1(IgniteError::IGNITE_ERR_BINARY,
+                            "Field name cannot be NULL.", "typeId", typeId);
+                    }
+
+                    int32_t res = meta.Get()->GetFieldId(name);
+
+                    if (res == 0)
+                        res = ignite::binary::GetBinaryStringHashCode(name);
+
+                    if (res == 0)
+                    {
+                        IGNITE_ERROR_FORMATTED_2(IgniteError::IGNITE_ERR_BINARY,
+                            "Field ID for the field name is zero. Please, redefine GetFieldId()"
+                            " method for the type or change field name", "typeId", typeId, "fieldName", name);
+                    }
+
+                    return res;
+                }
+
+                virtual BinaryIdResolver* Clone() const
+                {
+                    return new MetadataBinaryIdResolver(*this);
+                }
+
+            private:
+                /** Metadata snapshot. */
+                SPSnap meta;
+            };
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/d44de994/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_object_header.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_object_header.h b/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_object_header.h
index 8e74de6..cc31fbe 100644
--- a/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_object_header.h
+++ b/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_object_header.h
@@ -230,6 +230,19 @@ namespace ignite
                 }
 
                 /**
+                 * Get footer length.
+                 *
+                 * @return Footer length.
+                 */
+                int32_t GetFooterLength() const
+                {
+                    if (!HasSchema())
+                        return 0;
+
+                    return GetLength() - GetSchemaOffset();
+                }
+
+                /**
                  * Get size of data without header and footer.
                  *
                  * @return Data length.

http://git-wip-us.apache.org/repos/asf/ignite/blob/d44de994/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_object_impl.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_object_impl.h b/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_object_impl.h
index 0eb0dd6..880fff2 100644
--- a/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_object_impl.h
+++ b/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_object_impl.h
@@ -25,8 +25,11 @@
 
 #include <stdint.h>
 
-#include <ignite/impl/interop/interop.h>
+#include <ignite/binary/binary_type.h>
+
+#include <ignite/impl/interop/interop_input_stream.h>
 #include <ignite/impl/binary/binary_reader_impl.h>
+#include <ignite/impl/binary/binary_type_manager.h>
 
 namespace ignite
 {
@@ -50,8 +53,17 @@ namespace ignite
                  *
                  * @param mem Binary object memory.
                  * @param start Object starting position in memory.
+                 * @param idRslvr ID resolver. If null metaMgr is going to be
+                 *  used to extract meta for the type.
+                 * @param metaMgr Metadata manager. Can be null if you are not
+                 *  going to access fields by name.
+                 */
+                BinaryObjectImpl(interop::InteropMemory& mem, int32_t start, BinaryIdResolver* idRslvr, BinaryTypeManager* metaMgr);
+
+                /**
+                 * Destructor.
                  */
-                BinaryObjectImpl(interop::InteropMemory& mem, int32_t start);
+                ~BinaryObjectImpl();
 
                 /**
                  * Copy constructor.
@@ -69,25 +81,24 @@ namespace ignite
                 BinaryObjectImpl& operator=(const BinaryObjectImpl& other);
 
                 /**
-                 * Create from InteropMemory instance.
-                 * @throw IgniteError if the memory at the specified offset
-                 *    is not a valid BinaryObject.
+                 * Swap contents with another instance.
                  *
-                 * @param mem Memory.
-                 * @param offset Offset in memory.
-                 * @return BinaryObjectImpl instance.
+                 * @param other Another instance.
                  */
-                static BinaryObjectImpl FromMemory(interop::InteropMemory& mem, int32_t offset);
+                void Swap(BinaryObjectImpl& other);
 
                 /**
                  * Create from InteropMemory instance.
-                 * @warning Does not check memory for validity.
+                 * @throw IgniteError if the memory at the specified offset
+                 *    is not a valid BinaryObject.
                  *
                  * @param mem Memory.
                  * @param offset Offset in memory.
+                 * @param metaMgr Metadata manager. Can be null if you are not
+                 *  going to access fields by name.
                  * @return BinaryObjectImpl instance.
                  */
-                static BinaryObjectImpl FromMemoryUnsafe(interop::InteropMemory& mem, int32_t offset);
+                static BinaryObjectImpl FromMemory(interop::InteropMemory& mem, int32_t offset, BinaryTypeManager* metaMgr);
 
                 /**
                  * Deserialize object.
@@ -98,15 +109,69 @@ namespace ignite
                 template<typename T>
                 T Deserialize() const
                 {
+                    ignite::binary::BinaryType<T> bt;
+                    int32_t actualTypeId = GetTypeId();
+                    int32_t requestedTypeId = bt.GetTypeId();
+
+                    if (requestedTypeId != actualTypeId)
+                    {
+                        IGNITE_ERROR_FORMATTED_3(ignite::IgniteError::IGNITE_ERR_BINARY,
+                            "Trying to deserialize binary object to a different type", "memPtr", mem->PointerLong(),
+                            "actualTypeId", actualTypeId, "requestedTypeId", requestedTypeId);
+                    }
+
                     interop::InteropInputStream stream(mem);
 
                     stream.Position(start);
                     BinaryReaderImpl reader(&stream);
 
-                    return reader.ReadObject<T>();
+                    return reader.ReadTopObject<T>();
+                }
+
+                /**
+                 * Get field.
+                 * @throw IgniteError if the there is no specified field or if it
+                 *     is not of the specified type.
+                 *
+                 * @param name Field name.
+                 * @return Field value.
+                 */
+                template<typename T>
+                T GetField(const char* name) const
+                {
+                    CheckIdResolver();
+
+                    int32_t fieldId = idRslvr->GetFieldId(GetTypeId(), name);
+                    int32_t pos = FindField(fieldId);
+
+                    if (pos == -1)
+                        return T();
+
+                    interop::InteropInputStream stream(mem);
+
+                    stream.Position(pos);
+                    BinaryReaderImpl reader(&stream);
+
+                    return reader.ReadTopObject<T>();
                 }
 
                 /**
+                 * Check if the binary object has the specified field.
+                 *
+                 * @param name Field name.
+                 * @return True if the binary object has the specified field and
+                 *     false otherwise.
+                 */
+                bool HasField(const char* name) const;
+
+                /**
+                 * Gets the value of underlying enum in int form.
+                 *
+                 * @return The value of underlying enum in int form.
+                 */
+                int32_t GetEnumValue() const;
+
+                /**
                  * Get binary object field.
                  *
                  * @warning Works only if all object fields are objects.
@@ -133,6 +198,14 @@ namespace ignite
                 int32_t GetLength() const;
 
                 /**
+                 * Get object hash code.
+                 * @throw IgniteError if the object is not in a valid state.
+                 *
+                 * @return Object hash code.
+                 */
+                int32_t GetHashCode() const;
+
+                /**
                  * Get type ID.
                  * @throw IgniteError if the object is not in a valid state.
                  *
@@ -140,21 +213,38 @@ namespace ignite
                  */
                 int32_t GetTypeId() const;
 
+            private:
                 /**
-                 * Get object hash code.
-                 * @throw IgniteError if the object is not in a valid state.
+                 * Find field position in memory.
                  *
-                 * @return Object hash code.
+                 * @param fieldId Field Identifier.
+                 * @return Field position on success and negative value on failure.
                  */
-                int32_t GetHashCode() const;
+                int32_t FindField(const int32_t fieldId) const;
+
+                /**
+                 * Checks that id resolver is set.
+                 *
+                 * @throw IgniteError if idRslvr is not set.
+                 */
+                void CheckIdResolver() const;
 
-            private:
                 /** Underlying object memory. */
                 interop::InteropMemory* mem;
 
                 /** Object starting position in memory. */
                 int32_t start;
+
+                /** ID resolver. */
+                mutable BinaryIdResolver* idRslvr;
+
+                /** Type manager. */
+                BinaryTypeManager* metaMgr;
             };
+
+            /* Specialization */
+            template<>
+            BinaryObjectImpl BinaryObjectImpl::GetField(const char* name) const;
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/d44de994/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_schema.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_schema.h b/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_schema.h
index cf97f8d..c42adef 100644
--- a/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_schema.h
+++ b/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_schema.h
@@ -22,17 +22,19 @@
 #include <stdint.h>
 
 #include <ignite/common/common.h>
-#include <ignite/impl/interop/interop_output_stream.h>
 
 namespace ignite
 {
     namespace impl
     {
-        namespace binary
+        namespace interop
         {
-            /** Binary writer implementation forward declaration. */
-            class BinaryWriterImpl;
+            /* Forward declaration */
+            class InteropOutputStream;
+        }
 
+        namespace binary
+        {
             /**
              * Schema size variants.
              */

http://git-wip-us.apache.org/repos/asf/ignite/blob/d44de994/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_type_handler.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_type_handler.h b/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_type_handler.h
index d06e5d4..bb0e11c 100644
--- a/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_type_handler.h
+++ b/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_type_handler.h
@@ -31,20 +31,16 @@ namespace ignite
             /**
              * Type handler. Tracks all type updates during write session.
              */
-            class BinaryTypeHandler 
+            class BinaryTypeHandler
             {
             public:
+
                 /**
                  * Constructor.
                  *
                  * @param snap Snapshot.
                  */
                 BinaryTypeHandler(SPSnap snap);
-                
-                /**
-                 * Destructor.
-                 */
-                ~BinaryTypeHandler();
 
                 /**
                  * Callback invoked when field is being written.
@@ -56,42 +52,31 @@ namespace ignite
                 void OnFieldWritten(int32_t fieldId, std::string fieldName, int32_t fieldTypeId);
 
                 /**
-                 * Get initial snapshot.
-                 *
-                 * @param Snapshot.
-                 */
-                SPSnap GetSnapshot();
-
-                /**
                  * Whether any difference exists.
                  *
-                 * @param True if difference exists.
-                 */
-                bool HasDifference();
-
-                /**
-                 * Get recorded field IDs difference.
-                 *
-                 * @param Recorded field IDs difference.
+                 * @return True if difference exists.
                  */
-                std::set<int32_t>* GetFieldIds();
+                bool HasUpdate() const
+                {
+                    return updated.Get() != 0;
+                }
 
                 /**
-                 * Get recorded fields difference.
+                 * Get updated snapshot.
                  *
-                 * @param Recorded fields difference.
+                 * @return Updated snapshot.
                  */
-                std::map<std::string, int32_t>* GetFields();
+                SPSnap GetUpdated()
+                {
+                    return updated;
+                }
 
             private:
                 /** Snapshot. */
-                SPSnap snap;                          
+                SPSnap origin;
 
-                /** Recorded field IDs difference. */
-                std::set<int32_t>* fieldIds;           
-                
-                /** Recorded fields difference. */
-                std::map<std::string, int32_t>* fields; 
+                /** Snapshot. */
+                SPSnap updated;
 
                 IGNITE_NO_COPY_ASSIGNMENT(BinaryTypeHandler)
             };

http://git-wip-us.apache.org/repos/asf/ignite/blob/d44de994/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 63853f8..01538b8 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
@@ -49,25 +49,24 @@ namespace ignite
                 /**
                  * Get handler.
                  *
+                 * @param typeName Type name.
                  * @param typeId Type ID.
                  */
-                ignite::common::concurrent::SharedPointer<BinaryTypeHandler> GetHandler(int32_t typeId);
+                ignite::common::concurrent::SharedPointer<BinaryTypeHandler> GetHandler(const std::string& typeName, int32_t typeId);
 
                 /**
                  * Submit handler for processing.
-                 * 
-                 * @param typeName Type name.
-                 * @param typeId Type ID.
+                 *
                  * @param hnd Handler.
                  */
-                void SubmitHandler(std::string typeName, int32_t typeId, BinaryTypeHandler* hnd);
+                void SubmitHandler(BinaryTypeHandler& hnd);
 
                 /**
                  * Get current type manager version.
                  *
                  * @return Version.
                  */
-                int32_t GetVersion();
+                int32_t GetVersion() const;
 
                 /**
                  * Check whether something is updated since the given version.
@@ -75,16 +74,33 @@ namespace ignite
                  * @param oldVer Old version.
                  * @return True if updated and it is very likely that pending type exists.
                  */
-                bool IsUpdatedSince(int32_t oldVer);
+                bool IsUpdatedSince(int32_t oldVer) const;
 
                 /**
                  * Process pending updates.
                  *
-                 * @param updater Updater.
                  * @param err Error.
                  * @return In case of success.
                  */
-                bool ProcessPendingUpdates(BinaryTypeUpdater* updater, IgniteError* err);
+                bool ProcessPendingUpdates(IgniteError& err);
+
+                /**
+                 * Set updater.
+                 *
+                 * @param updater Updater.
+                 */
+                void SetUpdater(BinaryTypeUpdater* updater)
+                {
+                    this->updater = updater;
+                }
+
+                /**
+                 * Get metadata snapshop for the type.
+                 *
+                 * @param typeId Type ID.
+                 * @return Metadata snapshot.
+                 */
+                SPSnap GetMeta(int32_t typeId);
 
             private:
                 /** Current snapshots. */
@@ -94,7 +110,10 @@ namespace ignite
                 std::vector<SPSnap>* pending;
 
                 /** Critical section. */
-                ignite::common::concurrent::CriticalSection* cs;
+                ignite::common::concurrent::CriticalSection cs;
+
+                /** Type updater */
+                BinaryTypeUpdater* updater;
 
                 /** Version of pending changes. */
                 int32_t pendingVer;
@@ -103,15 +122,6 @@ namespace ignite
                 int32_t ver;
 
                 IGNITE_NO_COPY_ASSIGNMENT(BinaryTypeManager);
-
-                /**
-                 * Copy fields from a snapshot into relevant collections.
-                 *
-                 * @param snap Target snapshot.
-                 * @param fieldIds Field IDs.
-                 * @param fields Fields.
-                 */
-                void CopyFields(Snap* snap, std::set<int32_t>* fieldIds, std::map<std::string, int32_t>* fields);
             };
         }
     }    

http://git-wip-us.apache.org/repos/asf/ignite/blob/d44de994/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_type_snapshot.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_type_snapshot.h b/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_type_snapshot.h
index f1959fc..186bb84 100644
--- a/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_type_snapshot.h
+++ b/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_type_snapshot.h
@@ -25,6 +25,7 @@
 
 #include <ignite/common/common.h>
 #include <ignite/common/concurrent.h>
+#include <ignite/impl/binary/binary_field_meta.h>
 
 namespace ignite
 {
@@ -38,21 +39,23 @@ namespace ignite
             class BinaryTypeSnapshot
             {
             public:
+                typedef std::map<std::string, BinaryFieldMeta> FieldMap;
+                typedef std::set<int32_t> FieldIdSet;
+
                 /**
                  * Constructor.
                  *
                  * @param typeName Type name.
                  * @param typeId Type ID.
-                 * @param fieldIds Field IDs.
-                 * @param fields Fields.
                  */
-                BinaryTypeSnapshot(std::string typeName, int32_t typeId, std::set<int32_t>* fieldIds, 
-                    std::map<std::string, int32_t>* fields);
+                BinaryTypeSnapshot(std::string typeName, int32_t typeId);
 
                 /**
-                 * Destructor.
+                 * Copy constructor.
+                 *
+                 * @param another Another instance.
                  */
-                ~BinaryTypeSnapshot();
+                BinaryTypeSnapshot(const BinaryTypeSnapshot& another);
 
                 /**
                  * Check whether snapshot contains a field with the given ID.
@@ -60,42 +63,79 @@ namespace ignite
                  * @param fieldId Field ID.
                  * @return True if contains, false otherwise.
                  */
-                bool ContainsFieldId(int32_t fieldId);
+                bool ContainsFieldId(int32_t fieldId) const
+                {
+                    return fieldIds.count(fieldId) == 1;
+                }
 
                 /**
                  * Get type name.
                  *
                  * @return Type name.
                  */
-                std::string GetTypeName();
+                const std::string& GetTypeName() const
+                {
+                    return typeName;
+                }
 
                 /**
                  * Get type ID.
                  *
                  * @return Type ID.
                  */
-                int32_t GetTypeId();
+                int32_t GetTypeId() const
+                {
+                    return typeId;
+                }
 
                 /**
                  * Whether snapshot contains any fields.
                  *
                  * @return True if fields exist.
                  */
-                bool HasFields();
+                bool HasFields() const
+                {
+                    return !fieldIds.empty();
+                }
 
-                /** 
-                 * Get field IDs.
+                /**
+                 * Get field map.
                  *
-                 * @return Field IDs.
+                 * @return Fields.
                  */
-                std::set<int32_t>* GetFieldIds();
+                const FieldMap& GetFieldMap() const
+                {
+                    return fields;
+                }
 
                 /**
-                 * Get fields.
+                 * Add field meta.
                  *
-                 * @return Fields.
+                 * @param fieldId Field ID.
+                 * @param fieldName Field name.
+                 * @param fieldTypeId Field type ID.
+                 */
+                void AddField(int32_t fieldId, const std::string& fieldName, int32_t fieldTypeId);
+
+                /**
+                 * Copy fields from another snapshot.
+                 *
+                 * @param another Another instance.
+                 */
+                void CopyFieldsFrom(const BinaryTypeSnapshot* another);
+
+                /**
+                 * Get field ID.
+                 *
+                 * @param fieldName Field name.
+                 * @return Field ID on success and 0 on fail. 
                  */
-                std::map<std::string, int32_t>* GetFields();
+                int32_t GetFieldId(const std::string& fieldName)
+                {
+                    const FieldMap::const_iterator it = fields.find(fieldName);
+
+                    return it == fields.end() ? 0 : it->second.GetFieldId();
+                }
 
             private:
                 /** Type name. */
@@ -105,12 +145,10 @@ namespace ignite
                 int32_t typeId;
 
                 /** Known field IDs. */
-                std::set<int32_t>* fieldIds;
-
-                /** Field name-type mappings. */
-                std::map<std::string, int32_t>* fields; 
+                FieldIdSet fieldIds;
 
-                IGNITE_NO_COPY_ASSIGNMENT(BinaryTypeSnapshot)
+                /** Fields metadata. */
+                FieldMap fields;
             };
 
             typedef BinaryTypeSnapshot Snap;

http://git-wip-us.apache.org/repos/asf/ignite/blob/d44de994/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_type_updater.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_type_updater.h b/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_type_updater.h
index 6bded74..7a2c301 100644
--- a/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_type_updater.h
+++ b/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_type_updater.h
@@ -19,7 +19,7 @@
 #define _IGNITE_IMPL_BINARY_BINARY_TYPE_UPDATER
 
 #include <ignite/ignite_error.h>
-#include "ignite/impl/binary/binary_type_snapshot.h"
+#include <ignite/impl/binary/binary_type_snapshot.h>
 
 namespace ignite
 {    
@@ -36,15 +36,28 @@ namespace ignite
                 /**
                  * Destructor.
                  */
-                virtual ~BinaryTypeUpdater();
+                virtual ~BinaryTypeUpdater()
+                {
+                    // No-op.
+                }
 
                 /**
                  * Update type using provided snapshot.
                  *
                  * @param snapshot Snapshot.
                  * @param err Error.
+                 * @return True on success.
                  */
-                virtual bool Update(Snap* snapshot, IgniteError& err) = 0;
+                virtual bool Update(const Snap& snapshot, IgniteError& err) = 0;
+
+                /**
+                 * Get schema for type.
+                 *
+                 * @param typeId Type ID.
+                 * @param err Error.
+                 * @return Result.
+                 */
+                virtual SPSnap GetMeta(int32_t typeId, IgniteError& err) = 0;
             };
         }
     }    

http://git-wip-us.apache.org/repos/asf/ignite/blob/d44de994/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_writer_impl.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_writer_impl.h b/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_writer_impl.h
index ea5f81f..2cc86e1 100644
--- a/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_writer_impl.h
+++ b/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_writer_impl.h
@@ -32,12 +32,12 @@
 #include "ignite/impl/binary/binary_utils.h"
 #include "ignite/impl/binary/binary_schema.h"
 #include "ignite/impl/binary/binary_type_impl.h"
+#include "ignite/impl/binary/binary_type_manager.h"
 #include "ignite/binary/binary_consts.h"
 #include "ignite/binary/binary_type.h"
 #include "ignite/guid.h"
 #include "ignite/date.h"
 #include "ignite/timestamp.h"
-#include "binary_type_manager.h"
 
 namespace ignite
 {
@@ -632,7 +632,7 @@ namespace ignite
                  *
                  * @param id Session ID.
                  */
-                void CommitContainer(int32_t id);                
+                void CommitContainer(int32_t id);
 
                 /**
                  * Write object.
@@ -691,7 +691,7 @@ namespace ignite
                         ignite::common::concurrent::SharedPointer<BinaryTypeHandler> metaHnd;
 
                         if (metaMgr)
-                            metaHnd = metaMgr->GetHandler(idRslvr.GetTypeId());
+                            metaHnd = metaMgr->GetHandler(type.GetTypeName(), idRslvr.GetTypeId());
 
                         int32_t pos = stream->Position();
 
@@ -714,11 +714,14 @@ namespace ignite
 
                         stream->Synchronize();
 
-                        ignite::binary::BinaryObject binObj(*stream->GetMemory(), pos);
-                        stream->WriteInt32(hashPos, impl::binary::GetHashCode<T>(obj, binObj));
-
                         if (metaMgr)
-                            metaMgr->SubmitHandler(type.GetTypeName(), idRslvr.GetTypeId(), metaHnd.Get());
+                            metaMgr->SubmitHandler(*metaHnd.Get());
+
+                        // We are using direct constructor here to avoid check-overhead, as we know
+                        // at this point that underlying memory contains valid binary object.
+                        ignite::binary::BinaryObject binObj(*stream->GetMemory(), pos, &idRslvr, metaMgr);
+
+                        stream->WriteInt32(hashPos, impl::binary::GetHashCode<T>(obj, binObj));
                     }
                 }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/d44de994/modules/platforms/cpp/binary/project/vs/binary.vcxproj
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/binary/project/vs/binary.vcxproj b/modules/platforms/cpp/binary/project/vs/binary.vcxproj
index 6c1c433..5a01465 100644
--- a/modules/platforms/cpp/binary/project/vs/binary.vcxproj
+++ b/modules/platforms/cpp/binary/project/vs/binary.vcxproj
@@ -189,6 +189,7 @@
     <ClInclude Include="..\..\include\ignite\binary\binary_type.h" />
     <ClInclude Include="..\..\include\ignite\binary\binary_writer.h" />
     <ClInclude Include="..\..\include\ignite\impl\binary\binary_common.h" />
+    <ClInclude Include="..\..\include\ignite\impl\binary\binary_field_meta.h" />
     <ClInclude Include="..\..\include\ignite\impl\binary\binary_id_resolver.h" />
     <ClInclude Include="..\..\include\ignite\impl\binary\binary_object_header.h" />
     <ClInclude Include="..\..\include\ignite\impl\binary\binary_object_impl.h" />
@@ -215,6 +216,7 @@
     <ClCompile Include="..\..\src\binary\binary_reader.cpp" />
     <ClCompile Include="..\..\src\binary\binary_type.cpp" />
     <ClCompile Include="..\..\src\binary\binary_writer.cpp" />
+    <ClCompile Include="..\..\src\impl\binary\binary_field_meta.cpp" />
     <ClCompile Include="..\..\src\impl\binary\binary_object_header.cpp" />
     <ClCompile Include="..\..\src\impl\binary\binary_object_impl.cpp" />
     <ClCompile Include="..\..\src\impl\binary\binary_reader_impl.cpp" />
@@ -222,7 +224,6 @@
     <ClCompile Include="..\..\src\impl\binary\binary_type_handler.cpp" />
     <ClCompile Include="..\..\src\impl\binary\binary_type_manager.cpp" />
     <ClCompile Include="..\..\src\impl\binary\binary_type_snapshot.cpp" />
-    <ClCompile Include="..\..\src\impl\binary\binary_type_updater.cpp" />
     <ClCompile Include="..\..\src\impl\binary\binary_utils.cpp" />
     <ClCompile Include="..\..\src\impl\binary\binary_writer_impl.cpp" />
     <ClCompile Include="..\..\src\impl\interop\interop_input_stream.cpp" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/d44de994/modules/platforms/cpp/binary/project/vs/binary.vcxproj.filters
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/binary/project/vs/binary.vcxproj.filters b/modules/platforms/cpp/binary/project/vs/binary.vcxproj.filters
index c649c4d..05a21aa 100644
--- a/modules/platforms/cpp/binary/project/vs/binary.vcxproj.filters
+++ b/modules/platforms/cpp/binary/project/vs/binary.vcxproj.filters
@@ -106,6 +106,9 @@
     <ClInclude Include="..\..\include\ignite\impl\binary\binary_type_impl.h">
       <Filter>Code\impl\binary</Filter>
     </ClInclude>
+    <ClInclude Include="..\..\include\ignite\impl\binary\binary_field_meta.h">
+      <Filter>Code\impl\binary</Filter>
+    </ClInclude>
   </ItemGroup>
   <ItemGroup>
     <ClCompile Include="..\..\src\binary\binary_containers.cpp">
@@ -141,9 +144,6 @@
     <ClCompile Include="..\..\src\impl\binary\binary_type_snapshot.cpp">
       <Filter>Code\impl\binary</Filter>
     </ClCompile>
-    <ClCompile Include="..\..\src\impl\binary\binary_type_updater.cpp">
-      <Filter>Code\impl\binary</Filter>
-    </ClCompile>
     <ClCompile Include="..\..\src\impl\binary\binary_utils.cpp">
       <Filter>Code\impl\binary</Filter>
     </ClCompile>
@@ -168,5 +168,8 @@
     <ClCompile Include="..\..\src\binary\binary_array_identity_resolver.cpp">
       <Filter>Code\binary</Filter>
     </ClCompile>
+    <ClCompile Include="..\..\src\impl\binary\binary_field_meta.cpp">
+      <Filter>Code\impl\binary</Filter>
+    </ClCompile>
   </ItemGroup>
 </Project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/d44de994/modules/platforms/cpp/binary/src/impl/binary/binary_field_meta.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/binary/src/impl/binary/binary_field_meta.cpp b/modules/platforms/cpp/binary/src/impl/binary/binary_field_meta.cpp
new file mode 100644
index 0000000..3d0b065
--- /dev/null
+++ b/modules/platforms/cpp/binary/src/impl/binary/binary_field_meta.cpp
@@ -0,0 +1,42 @@
+/*
+ * 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.
+ */
+
+ #include <ignite/binary/binary_raw_reader.h>
+ #include <ignite/binary/binary_raw_writer.h>
+
+#include <ignite/impl/binary/binary_field_meta.h>
+
+namespace ignite
+{
+    namespace impl
+    {
+        namespace binary
+        {
+            void BinaryFieldMeta::Write(ignite::binary::BinaryRawWriter& writer) const
+            {
+                writer.WriteInt32(typeId);
+                writer.WriteInt32(fieldId);
+            }
+
+            void BinaryFieldMeta::Read(ignite::binary::BinaryRawReader& reader)
+            {
+                typeId = reader.ReadInt32();
+                fieldId = reader.ReadInt32();
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/d44de994/modules/platforms/cpp/binary/src/impl/binary/binary_object_impl.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/binary/src/impl/binary/binary_object_impl.cpp b/modules/platforms/cpp/binary/src/impl/binary/binary_object_impl.cpp
index 0b397b1..38e7670 100644
--- a/modules/platforms/cpp/binary/src/impl/binary/binary_object_impl.cpp
+++ b/modules/platforms/cpp/binary/src/impl/binary/binary_object_impl.cpp
@@ -15,8 +15,12 @@
  * limitations under the License.
  */
 
+#include <ignite/impl/interop/interop_stream_position_guard.h>
+#include <ignite/impl/interop/interop_input_stream.h>
+
 #include <ignite/impl/binary/binary_object_header.h>
 #include <ignite/impl/binary/binary_object_impl.h>
+#include <ignite/impl/binary/binary_utils.h>
 
 using namespace ignite::impl::binary;
 
@@ -26,28 +30,54 @@ namespace ignite
     {
         namespace binary
         {
-            BinaryObjectImpl::BinaryObjectImpl(interop::InteropMemory& mem, int32_t start) :
+            BinaryObjectImpl::BinaryObjectImpl(interop::InteropMemory& mem,
+                int32_t start, BinaryIdResolver* idRslvr, BinaryTypeManager* metaMgr) :
                 mem(&mem),
-                start(start)
+                start(start),
+                idRslvr(0),
+                metaMgr(metaMgr)
+            {
+                if (idRslvr)
+                    this->idRslvr = idRslvr->Clone();
+            }
+
+            BinaryObjectImpl::~BinaryObjectImpl()
             {
-                // No-op.
+                delete idRslvr;
             }
 
             BinaryObjectImpl::BinaryObjectImpl(const BinaryObjectImpl& other) :
                 mem(other.mem),
-                start(other.start)
+                start(other.start),
+                idRslvr(0),
+                metaMgr(other.metaMgr)
             {
-                // No-op.
+                if (other.idRslvr)
+                    this->idRslvr = other.idRslvr->Clone();
             }
 
             BinaryObjectImpl& BinaryObjectImpl::operator=(const BinaryObjectImpl& other)
             {
-                mem = other.mem;
+                if (this != &other)
+                {
+                    BinaryObjectImpl tmp(other);
+
+                    Swap(tmp);
+                }
 
                 return *this;
             }
 
-            BinaryObjectImpl BinaryObjectImpl::FromMemory(interop::InteropMemory& mem, int32_t offset)
+            void BinaryObjectImpl::Swap(BinaryObjectImpl& other)
+            {
+                std::swap(mem, other.mem);
+                std::swap(start, other.start);
+                std::swap(idRslvr, other.idRslvr);
+                std::swap(metaMgr, other.metaMgr);
+            }
+
+            BinaryObjectImpl BinaryObjectImpl::FromMemory(interop::InteropMemory& mem, int32_t offset,
+                BinaryTypeManager* metaMgr)
             {
                 BinaryObjectHeader header = BinaryObjectHeader::FromMemory(mem, offset);
 
@@ -55,12 +85,34 @@ namespace ignite
 
                 assert(adjustedStart >= 0);
 
-                return BinaryObjectImpl(mem, adjustedStart);
+                return BinaryObjectImpl(mem, adjustedStart, 0, metaMgr);
             }
 
-            BinaryObjectImpl BinaryObjectImpl::FromMemoryUnsafe(interop::InteropMemory& mem, int32_t offset)
+            template<>
+            BinaryObjectImpl BinaryObjectImpl::GetField(const char* name) const
             {
-                return BinaryObjectImpl(mem, offset);
+                CheckIdResolver();
+
+                int32_t fieldId = idRslvr->GetFieldId(GetTypeId(), name);
+                int32_t pos = FindField(fieldId);
+
+                return FromMemory(*mem, pos, metaMgr);
+            }
+
+            bool BinaryObjectImpl::HasField(const char* name) const
+            {
+                CheckIdResolver();
+
+                int32_t fieldId = idRslvr->GetFieldId(GetTypeId(), name);
+
+                int32_t fieldPos = FindField(fieldId);
+
+                return fieldPos >= 0;
+            }
+
+            int32_t BinaryObjectImpl::GetEnumValue() const
+            {
+                throw IgniteError(IgniteError::IGNITE_ERR_BINARY, "GetEnumValue is only supported for enums.");
             }
 
             BinaryObjectImpl BinaryObjectImpl::GetField(int32_t idx)
@@ -74,7 +126,7 @@ namespace ignite
                     offset += fieldHeader.GetLength();
                 }
 
-                return BinaryObjectImpl::FromMemory(*mem, offset);
+                return FromMemory(*mem, offset, 0);
             }
 
             const int8_t* BinaryObjectImpl::GetData() const
@@ -102,6 +154,71 @@ namespace ignite
 
                 return header.GetHashCode();
             }
+
+            int32_t BinaryObjectImpl::FindField(const int32_t fieldId) const
+            {
+                BinaryObjectHeader header(mem->Data() + start);
+                int32_t flags = header.GetFlags();
+
+                int32_t footerBegin = header.GetFooterOffset() + start;
+                int32_t footerEnd = footerBegin + header.GetFooterLength();
+
+                if ((mem->Length() - start) < footerEnd)
+                {
+                    IGNITE_ERROR_FORMATTED_3(ignite::IgniteError::IGNITE_ERR_MEMORY,
+                        "Not enough data in the binary object", "memPtr", mem->PointerLong(),
+                        "len", (mem->Length() - start), "footerEnd", footerEnd);
+                }
+
+                if (flags & IGNITE_BINARY_FLAG_OFFSET_ONE_BYTE)
+                {
+                    for (int32_t schemaPos = footerBegin; schemaPos < footerEnd; schemaPos += 5)
+                    {
+                        int32_t currentFieldId = BinaryUtils::UnsafeReadInt32(*mem, schemaPos);
+
+                        if (fieldId == currentFieldId)
+                            return (BinaryUtils::UnsafeReadInt8(*mem, schemaPos + 4) & 0xFF) + start;
+                    }
+                }
+                else if (flags & IGNITE_BINARY_FLAG_OFFSET_TWO_BYTES)
+                {
+                    for (int32_t schemaPos = footerBegin; schemaPos < footerEnd; schemaPos += 6)
+                    {
+                        int32_t currentFieldId = BinaryUtils::UnsafeReadInt32(*mem, schemaPos);
+
+                        if (fieldId == currentFieldId)
+                            return (BinaryUtils::UnsafeReadInt16(*mem, schemaPos + 4) & 0xFFFF) + start;
+                    }
+                }
+                else
+                {
+                    for (int32_t schemaPos = footerBegin; schemaPos < footerEnd; schemaPos += 8)
+                    {
+                        int32_t currentFieldId = BinaryUtils::UnsafeReadInt32(*mem, schemaPos);
+
+                        if (fieldId == currentFieldId)
+                            return BinaryUtils::UnsafeReadInt32(*mem, schemaPos + 4) + start;
+                    }
+                }
+
+                return -1;
+            }
+
+            void BinaryObjectImpl::CheckIdResolver() const
+            {
+                if (idRslvr)
+                    return;
+
+                assert(metaMgr != 0);
+
+                BinaryObjectHeader header(mem->Data() + start);
+
+                int32_t typeId = header.GetTypeId();
+
+                SPSnap meta = metaMgr->GetMeta(typeId);
+
+                idRslvr = new MetadataBinaryIdResolver(meta);
+            }
         }
     }
 }
\ No newline at end of file