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 2015/10/30 16:52:36 UTC

[40/52] [abbrv] ignite git commit: IGNITE-1770: Implemented constant-time field lookup on protocol level.

http://git-wip-us.apache.org/repos/asf/ignite/blob/b85fa171/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableObjectSchema.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableObjectSchema.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableObjectSchema.cs
new file mode 100644
index 0000000..7d3663c
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableObjectSchema.cs
@@ -0,0 +1,98 @@
+/*
+ * 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.Portable
+{
+    using System.Collections.Generic;
+
+    /// <summary>
+    /// Holds and manages portable object schema for a specific type.
+    /// </summary>
+    internal class PortableObjectSchema
+    {
+        /** First schema id. */
+        private volatile int _schemaId1;
+
+        /** First schema. */
+        private volatile int[] _schema1;
+
+        /** Second schema id. */
+        private volatile int _schemaId2;
+
+        /** Second schema. */
+        private volatile int[] _schema2;
+
+        /** Other schemas. */
+        private volatile Dictionary<int, int[]> _schemas;
+
+        /// <summary>
+        /// Gets the schema by id.
+        /// </summary>
+        /// <param name="id">Schema id.</param>
+        /// <returns>Schema or null.</returns>
+        public int[] GetSchema(int id)
+        {
+            if (_schemaId1 == id)
+                return _schema1;
+
+            if (_schemaId2 == id)
+                return _schema2;
+
+            int[] res;
+
+            if (_schemas != null && _schemas.TryGetValue(id, out res))
+                return res;
+
+            return null;
+        }
+
+        /// <summary>
+        /// Adds the schema.
+        /// </summary>
+        /// <param name="id">Schema id.</param>
+        /// <param name="schema">Schema.</param>
+        public void AddSchema(int id, int[] schema)
+        {
+            lock (this)
+            {
+                if (_schemaId1 == id || _schemaId2 == id || (_schemas != null && _schemas.ContainsKey(id)))
+                    return;
+
+                if (_schema1 == null)
+                {
+                    _schemaId1 = id;
+                    _schema1 = schema;
+                }
+                else if (_schema2 == null)
+                {
+                    _schemaId2 = id;
+                    _schema2 = schema;
+                }
+                else
+                {
+                    var schemas = _schemas == null 
+                        ? new Dictionary<int, int[]>() 
+                        : new Dictionary<int, int[]>(_schemas);
+
+                    schemas.Add(id, schema);
+
+                    _schemas = schemas;
+                }
+            }
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/b85fa171/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableObjectSchemaField.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableObjectSchemaField.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableObjectSchemaField.cs
new file mode 100644
index 0000000..48fd9c1
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableObjectSchemaField.cs
@@ -0,0 +1,113 @@
+/*
+ * 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.Portable
+{
+    using System;
+    using System.Diagnostics;
+    using System.Runtime.InteropServices;
+    using Apache.Ignite.Core.Impl.Portable.IO;
+
+    /// <summary>
+    /// Portable schema field DTO (as it is stored in a stream).
+    /// </summary>
+    [StructLayout(LayoutKind.Sequential)]
+    internal struct PortableObjectSchemaField
+    {
+        /* Field ID */
+        public readonly int Id;
+
+        /** Offset. */
+        public readonly int Offset;
+
+        /** Size, equals to sizeof(PortableObjectSchemaField) */
+        private const int Size = 8;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="PortableObjectSchemaField"/> struct.
+        /// </summary>
+        /// <param name="id">The id.</param>
+        /// <param name="offset">The offset.</param>
+        public PortableObjectSchemaField(int id, int offset)
+        {
+            Id = id;
+            Offset = offset;
+        }
+
+        /// <summary>
+        /// Writes an array of fields to a stream.
+        /// </summary>
+        /// <param name="fields">Fields.</param>
+        /// <param name="stream">Stream.</param>
+        /// <param name="count">Field count to write.</param>
+        public static unsafe void WriteArray(PortableObjectSchemaField[] fields, IPortableStream stream, int count)
+        {
+            Debug.Assert(fields != null);
+            Debug.Assert(stream != null);
+            Debug.Assert(count > 0);
+
+            if (BitConverter.IsLittleEndian)
+            {
+                fixed (PortableObjectSchemaField* ptr = &fields[0])
+                {
+                    stream.Write((byte*) ptr, count * Size);
+                }
+            }
+            else
+            {
+                for (int i = 0; i < count; i++)
+                {
+                    var field = fields[i];
+
+                    stream.WriteInt(field.Id);
+                    stream.WriteInt(field.Offset);
+                }
+            }
+        }
+
+        /// <summary>
+        /// Reads an array of fields from a stream.
+        /// </summary>
+        /// <param name="stream">Stream.</param>
+        /// <param name="count">Count.</param>
+        /// <returns></returns>
+        public static unsafe PortableObjectSchemaField[] ReadArray(IPortableStream stream, int count)
+        {
+            Debug.Assert(stream != null);
+            Debug.Assert(count > 0);
+
+            var res = new PortableObjectSchemaField[count];
+
+            if (BitConverter.IsLittleEndian)
+            {
+                fixed (PortableObjectSchemaField* ptr = &res[0])
+                {
+                    stream.Read((byte*) ptr, count * Size);
+                }
+            }
+            else
+            {
+                for (int i = 0; i < count; i++) 
+                {
+                    res[i] = new PortableObjectSchemaField(stream.ReadInt(), stream.ReadInt());
+                }
+            }
+
+            return res;
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/b85fa171/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableReaderImpl.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableReaderImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableReaderImpl.cs
index 422d628..48ea799 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableReaderImpl.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableReaderImpl.cs
@@ -66,6 +66,14 @@ namespace Apache.Ignite.Core.Impl.Portable
         /** Current type structure tracker. */
         private PortableStructureTracker _curStruct;
 
+        /** */
+        private int _curFooterStart;
+
+        /** */
+        private int _curFooterEnd;
+
+        /** */
+        private int[] _curSchema;
 
         /// <summary>
         /// Constructor.
@@ -554,10 +562,10 @@ namespace Apache.Ignite.Core.Impl.Portable
         {
             var len = Stream.ReadInt();
 
-            var portablePos = Stream.Position;
+            var portableBytesPos = Stream.Position;
 
             if (_mode != PortableMode.Deserialize)
-                return TypeCaster<T>.Cast(ReadAsPortable(portablePos, len, doDetach));
+                return TypeCaster<T>.Cast(ReadAsPortable(portableBytesPos, len, doDetach));
 
             Stream.Seek(len, SeekOrigin.Current);
 
@@ -565,7 +573,7 @@ namespace Apache.Ignite.Core.Impl.Portable
 
             var retPos = Stream.Position;
 
-            Stream.Seek(portablePos + offset, SeekOrigin.Begin);
+            Stream.Seek(portableBytesPos + offset, SeekOrigin.Begin);
 
             _mode = PortableMode.KeepPortable;
 
@@ -584,30 +592,28 @@ namespace Apache.Ignite.Core.Impl.Portable
         /// <summary>
         /// Reads the portable object in portable form.
         /// </summary>
-        private PortableUserObject ReadAsPortable(int dataPos, int dataLen, bool doDetach)
+        private PortableUserObject ReadAsPortable(int portableBytesPos, int dataLen, bool doDetach)
         {
             try
             {
-                Stream.Seek(dataLen + dataPos, SeekOrigin.Begin);
+                Stream.Seek(dataLen + portableBytesPos, SeekOrigin.Begin);
 
                 var offs = Stream.ReadInt(); // offset inside data
 
-                var pos = dataPos + offs;
+                var pos = portableBytesPos + offs;
 
-                if (!doDetach)
-                    return GetPortableUserObject(pos, pos, Stream.Array());
-                
-                Stream.Seek(pos + PortableUtils.OffsetLen, SeekOrigin.Begin);
+                var hdr = PortableObjectHeader.Read(Stream, pos);
 
-                var len = Stream.ReadInt();
+                if (!doDetach)
+                    return new PortableUserObject(_marsh, Stream.GetArray(), pos, hdr);
 
                 Stream.Seek(pos, SeekOrigin.Begin);
 
-                return GetPortableUserObject(pos, 0, Stream.ReadByteArray(len));
+                return new PortableUserObject(_marsh, Stream.ReadByteArray(hdr.Length), 0, hdr);
             }
             finally
             {
-                Stream.Seek(dataPos + dataLen + 4, SeekOrigin.Begin);
+                Stream.Seek(portableBytesPos + dataLen + 4, SeekOrigin.Begin);
             }
         }
 
@@ -617,16 +623,10 @@ namespace Apache.Ignite.Core.Impl.Portable
         [SuppressMessage("Microsoft.Performance", "CA1804:RemoveUnusedLocals", MessageId = "hashCode")]
         private T ReadFullObject<T>(int pos)
         {
-            // Validate protocol version.
-            PortableUtils.ValidateProtocolVersion(Stream);
+            var hdr = PortableObjectHeader.Read(Stream, pos);
 
-            // Read header.
-            bool userType = Stream.ReadBool();
-            int typeId = Stream.ReadInt();
-            // ReSharper disable once UnusedVariable
-            int hashCode = Stream.ReadInt();
-            int len = Stream.ReadInt();
-            int rawOffset = Stream.ReadInt();
+            // Validate protocol version.
+            PortableUtils.ValidateProtocolVersion(hdr.Version);
 
             try
             {
@@ -636,7 +636,7 @@ namespace Apache.Ignite.Core.Impl.Portable
                 if (_hnds != null && _hnds.TryGetValue(pos, out hndObj))
                     return (T) hndObj;
 
-                if (userType && _mode == PortableMode.ForcePortable)
+                if (hdr.IsUserType && _mode == PortableMode.ForcePortable)
                 {
                     PortableUserObject portObj;
 
@@ -644,10 +644,10 @@ namespace Apache.Ignite.Core.Impl.Portable
                     {
                         Stream.Seek(pos, SeekOrigin.Begin);
 
-                        portObj = GetPortableUserObject(pos, 0, Stream.ReadByteArray(len));
+                        portObj = new PortableUserObject(_marsh, Stream.ReadByteArray(hdr.Length), 0, hdr);
                     }
                     else
-                        portObj = GetPortableUserObject(pos, pos, Stream.Array());
+                        portObj = new PortableUserObject(_marsh, Stream.GetArray(), pos, hdr);
 
                     T obj = _builder == null ? TypeCaster<T>.Cast(portObj) : TypeCaster<T>.Cast(_builder.Child(portObj));
 
@@ -660,8 +660,8 @@ namespace Apache.Ignite.Core.Impl.Portable
                     // Find descriptor.
                     IPortableTypeDescriptor desc;
 
-                    if (!_descs.TryGetValue(PortableUtils.TypeKey(userType, typeId), out desc))
-                        throw new PortableException("Unknown type ID: " + typeId);
+                    if (!_descs.TryGetValue(PortableUtils.TypeKey(hdr.IsUserType, hdr.TypeId), out desc))
+                        throw new PortableException("Unknown type ID: " + hdr.TypeId);
 
                     // Instantiate object. 
                     if (desc.Type == null)
@@ -674,15 +674,32 @@ namespace Apache.Ignite.Core.Impl.Portable
                     int oldRawOffset = _curRawOffset;
                     var oldStruct = _curStruct;
                     bool oldRaw = _curRaw;
+                    var oldFooterStart = _curFooterStart;
+                    var oldFooterEnd = _curFooterEnd;
+                    var oldSchema = _curSchema;
 
                     // Set new frame.
-                    _curTypeId = typeId;
+                    _curTypeId = hdr.TypeId;
                     _curPos = pos;
-                    _curRawOffset = rawOffset;
+                    _curFooterEnd = hdr.GetSchemaEnd(pos);
+                    _curFooterStart = hdr.GetSchemaStart(pos);
+                    
+                    _curSchema = desc.Schema.GetSchema(hdr.SchemaId);
+
+                    if (_curSchema == null)
+                    {
+                        _curSchema = ReadSchema();
+
+                        desc.Schema.AddSchema(hdr.SchemaId, _curSchema);
+                    }
+
+                    _curRawOffset = hdr.GetRawOffset(Stream, pos);
                     _curStruct = new PortableStructureTracker(desc, desc.ReaderTypeStructure);
                     _curRaw = false;
 
                     // Read object.
+                    Stream.Seek(pos + PortableObjectHeader.Size, SeekOrigin.Begin);
+
                     object obj;
 
                     var sysSerializer = desc.Serializer as IPortableSystemTypeSerializer;
@@ -715,6 +732,9 @@ namespace Apache.Ignite.Core.Impl.Portable
                     _curRawOffset = oldRawOffset;
                     _curStruct = oldStruct;
                     _curRaw = oldRaw;
+                    _curFooterStart = oldFooterStart;
+                    _curFooterEnd = oldFooterEnd;
+                    _curSchema = oldSchema;
 
                     // Process wrappers. We could introduce a common interface, but for only 2 if-else is faster.
                     var wrappedSerializable = obj as SerializableObjectHolder;
@@ -733,11 +753,31 @@ namespace Apache.Ignite.Core.Impl.Portable
             finally
             {
                 // Advance stream pointer.
-                Stream.Seek(pos + len, SeekOrigin.Begin);
+                Stream.Seek(pos + hdr.Length, SeekOrigin.Begin);
             }
         }
 
         /// <summary>
+        /// Reads the schema.
+        /// </summary>
+        private int[] ReadSchema()
+        {
+            Stream.Seek(_curFooterStart, SeekOrigin.Begin);
+            
+            var count = (_curFooterEnd - _curFooterStart) >> 3;
+            
+            var res = new int[count];
+
+            for (int i = 0; i < count; i++)
+            {
+                res[i] = Stream.ReadInt();
+                Stream.Seek(4, SeekOrigin.Current);
+            }
+
+            return res;
+        }
+
+        /// <summary>
         /// Reads the handle object.
         /// </summary>
         private T ReadHandleObject<T>(int pos)
@@ -817,51 +857,21 @@ namespace Apache.Ignite.Core.Impl.Portable
         /// <returns>True in case the field was found and position adjusted, false otherwise.</returns>
         private bool SeekField(int fieldId)
         {
-            // This method is expected to be called when stream pointer is set either before
-            // the field or on raw data offset.
-            int start = _curPos + PortableUtils.FullHdrLen;
-            int end = _curPos + _curRawOffset;
-
-            int initial = Stream.Position;
+            Stream.Seek(_curFooterStart, SeekOrigin.Begin);
 
-            int cur = initial;
-
-            while (cur < end)
+            while (Stream.Position < _curFooterEnd)
             {
-                int id = Stream.ReadInt();
+                var id = Stream.ReadInt();
 
-                if (fieldId == id)
+                if (id == fieldId)
                 {
-                    // Field is found, return.
-                    Stream.Seek(4, SeekOrigin.Current);
+                    var fieldOffset = Stream.ReadInt();
 
+                    Stream.Seek(_curPos + fieldOffset, SeekOrigin.Begin);
                     return true;
                 }
-                
-                Stream.Seek(Stream.ReadInt(), SeekOrigin.Current);
 
-                cur = Stream.Position;
-            }
-
-            Stream.Seek(start, SeekOrigin.Begin);
-
-            cur = start;
-
-            while (cur < initial)
-            {
-                int id = Stream.ReadInt();
-
-                if (fieldId == id)
-                {
-                    // Field is found, return.
-                    Stream.Seek(4, SeekOrigin.Current);
-
-                    return true;
-                }
-                
-                Stream.Seek(Stream.ReadInt(), SeekOrigin.Current);
-
-                cur = Stream.Position;
+                Stream.Seek(4, SeekOrigin.Current);
             }
 
             return false;
@@ -892,11 +902,19 @@ namespace Apache.Ignite.Core.Impl.Portable
             if (_curRaw)
                 throw new PortableException("Cannot read named fields after raw data is read.");
 
+            var actionId = _curStruct.CurStructAction;
+
             var fieldId = _curStruct.GetFieldId(fieldName);
 
-            if (!SeekField(fieldId))
-                return false;
+            if (_curSchema == null || actionId >= _curSchema.Length || fieldId != _curSchema[actionId])
+            {
+                _curSchema = null;   // read order is different, ignore schema for future reads
+
+                if (!SeekField(fieldId))
+                    return false;
+            }
 
+            // Expected read order, no need to seek.
             return IsNotNullHeader(expHdr);
         }
 
@@ -939,22 +957,5 @@ namespace Apache.Ignite.Core.Impl.Portable
         {
             return IsNotNullHeader(expHdr) ? readFunc(Stream) : default(T);
         }
-
-        /// <summary>
-        /// Gets the portable user object from a byte array.
-        /// </summary>
-        /// <param name="pos">Position in the current stream.</param>
-        /// <param name="offs">Offset in the byte array.</param>
-        /// <param name="bytes">Bytes.</param>
-        private PortableUserObject GetPortableUserObject(int pos, int offs, byte[] bytes)
-        {
-            Stream.Seek(pos + PortableUtils.OffsetTypeId, SeekOrigin.Begin);
-
-            var id = Stream.ReadInt();
-
-            var hash = Stream.ReadInt();
-
-            return new PortableUserObject(_marsh, bytes, offs, id, hash);
-        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/b85fa171/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableSurrogateTypeDescriptor.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableSurrogateTypeDescriptor.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableSurrogateTypeDescriptor.cs
index 8560ce8..71361b4 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableSurrogateTypeDescriptor.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableSurrogateTypeDescriptor.cs
@@ -43,6 +43,9 @@ namespace Apache.Ignite.Core.Impl.Portable
 
         /** Type structure. */
         private PortableStructure _readerTypeStructure = PortableStructure.CreateEmpty();
+        
+        /** Type schema. */
+        private readonly PortableObjectSchema _schema = new PortableObjectSchema();
 
         /// <summary>
         /// Constructor.
@@ -156,5 +159,11 @@ namespace Apache.Ignite.Core.Impl.Portable
                 _readerTypeStructure = _readerTypeStructure.Merge(exp, pathIdx, updates);
             }
         }
+
+        /** <inheritDoc /> */
+        public PortableObjectSchema Schema
+        {
+            get { return _schema; }
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/b85fa171/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableUserObject.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableUserObject.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableUserObject.cs
index c241b96..300281b 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableUserObject.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableUserObject.cs
@@ -31,6 +31,9 @@ namespace Apache.Ignite.Core.Impl.Portable
     /// </summary>
     internal class PortableUserObject : IPortableObject
     {
+        /** Cache empty dictionary. */
+        private static readonly IDictionary<int, int> EmptyFields = new Dictionary<int, int>();
+
         /** Marshaller. */
         private readonly PortableMarshaller _marsh;
 
@@ -40,11 +43,8 @@ namespace Apache.Ignite.Core.Impl.Portable
         /** Offset in data array. */
         private readonly int _offset;
 
-        /** Type ID. */
-        private readonly int _typeId;
-
-        /** Hash code. */
-        private readonly int _hashCode;
+        /** Header. */
+        private readonly PortableObjectHeader _header;
 
         /** Fields. */
         private volatile IDictionary<int, int> _fields;
@@ -53,28 +53,26 @@ namespace Apache.Ignite.Core.Impl.Portable
         private object _deserialized;
 
         /// <summary>
-        /// Initializes a new instance of the <see cref="PortableUserObject"/> class.
+        /// Initializes a new instance of the <see cref="PortableUserObject" /> class.
         /// </summary>
         /// <param name="marsh">Marshaller.</param>
         /// <param name="data">Raw data of this portable object.</param>
         /// <param name="offset">Offset in data array.</param>
-        /// <param name="typeId">Type ID.</param>
-        /// <param name="hashCode">Hash code.</param>
-        public PortableUserObject(PortableMarshaller marsh, byte[] data, int offset, int typeId, int hashCode)
+        /// <param name="header">The header.</param>
+        public PortableUserObject(PortableMarshaller marsh, byte[] data, int offset, PortableObjectHeader header)
         {
             _marsh = marsh;
 
             _data = data;
             _offset = offset;
 
-            _typeId = typeId;
-            _hashCode = hashCode;
+            _header = header;
         }
 
         /** <inheritdoc /> */
         public int TypeId
         {
-            get { return _typeId; }
+            get { return _header.TypeId; }
         }
 
         /** <inheritdoc /> */
@@ -95,7 +93,7 @@ namespace Apache.Ignite.Core.Impl.Portable
         {
             IPortableStream stream = new PortableHeapStream(_data);
 
-            stream.Seek(pos, SeekOrigin.Begin);
+            stream.Seek(pos + _offset, SeekOrigin.Begin);
 
             return _marsh.Unmarshal<T>(stream, PortableMode.ForcePortable, builder);
         }
@@ -123,7 +121,7 @@ namespace Apache.Ignite.Core.Impl.Portable
 
                 T res = _marsh.Unmarshal<T>(stream, mode);
 
-                IPortableTypeDescriptor desc = _marsh.GetDescriptor(true, _typeId);
+                IPortableTypeDescriptor desc = _marsh.GetDescriptor(true, _header.TypeId);
 
                 if (!desc.KeepDeserialized)
                     return res;
@@ -137,7 +135,7 @@ namespace Apache.Ignite.Core.Impl.Portable
         /** <inheritdoc /> */
         public IPortableMetadata GetMetadata()
         {
-            return _marsh.GetMetadata(_typeId);
+            return _marsh.GetMetadata(_header.TypeId);
         }
 
         /// <summary>
@@ -158,11 +156,11 @@ namespace Apache.Ignite.Core.Impl.Portable
 
         public bool TryGetFieldPosition(string fieldName, out int pos)
         {
-            var desc = _marsh.GetDescriptor(true, _typeId);
+            var desc = _marsh.GetDescriptor(true, _header.TypeId);
 
             InitializeFields();
 
-            int fieldId = PortableUtils.FieldId(_typeId, fieldName, desc.NameConverter, desc.Mapper);
+            int fieldId = PortableUtils.FieldId(_header.TypeId, fieldName, desc.NameConverter, desc.Mapper);
 
             return _fields.TryGetValue(fieldId, out pos);
         }
@@ -172,22 +170,20 @@ namespace Apache.Ignite.Core.Impl.Portable
         /// </summary>
         private void InitializeFields()
         {
-            if (_fields == null)
-            {
-                IPortableStream stream = new PortableHeapStream(_data);
+            if (_fields != null) 
+                return;
 
-                stream.Seek(_offset + PortableUtils.OffsetRaw, SeekOrigin.Begin);
+            var stream = new PortableHeapStream(_data);
 
-                int rawDataOffset = stream.ReadInt();
+            var hdr = PortableObjectHeader.Read(stream, _offset);
 
-                _fields = PortableUtils.ObjectFields(stream, _typeId, rawDataOffset);
-            }
+            _fields = hdr.ReadSchemaAsDictionary(stream, _offset) ?? EmptyFields;
         }
 
         /** <inheritdoc /> */
         public override int GetHashCode()
         {
-            return _hashCode;
+            return _header.HashCode;
         }
 
         /** <inheritdoc /> */
@@ -203,8 +199,8 @@ namespace Apache.Ignite.Core.Impl.Portable
                 if (_data == that._data && _offset == that._offset)
                     return true;
 
-                // 1. Check hash code and type IDs.
-                if (_hashCode == that._hashCode && _typeId == that._typeId)
+                // 1. Check headers
+                if (_header == that._header)
                 {
                     // 2. Check if objects have the same field sets.
                     InitializeFields();
@@ -215,7 +211,7 @@ namespace Apache.Ignite.Core.Impl.Portable
 
                     foreach (int id in _fields.Keys)
                     {
-                        if (!that._fields.Keys.Contains(id))
+                        if (!that._fields.ContainsKey(id))
                             return false;
                     }
 
@@ -230,18 +226,16 @@ namespace Apache.Ignite.Core.Impl.Portable
                     }
 
                     // 4. Check if objects have the same raw data.
-                    IPortableStream stream = new PortableHeapStream(_data);
-                    stream.Seek(_offset + PortableUtils.OffsetLen, SeekOrigin.Begin);
-                    int len = stream.ReadInt();
-                    int rawOffset = stream.ReadInt();
-
-                    IPortableStream thatStream = new PortableHeapStream(that._data);
-                    thatStream.Seek(_offset + PortableUtils.OffsetLen, SeekOrigin.Begin);
-                    int thatLen = thatStream.ReadInt();
-                    int thatRawOffset = thatStream.ReadInt();
-
-                    return PortableUtils.CompareArrays(_data, _offset + rawOffset, len - rawOffset, that._data,
-                        that._offset + thatRawOffset, thatLen - thatRawOffset);
+                    // ReSharper disable ImpureMethodCallOnReadonlyValueField (method is not impure)
+                    var stream = new PortableHeapStream(_data);
+                    var rawOffset = _header.GetRawOffset(stream, _offset);
+
+                    var thatStream = new PortableHeapStream(that._data);
+                    var thatRawOffset = that._header.GetRawOffset(thatStream, that._offset);
+                    // ReSharper restore ImpureMethodCallOnReadonlyValueField
+
+                    return PortableUtils.CompareArrays(_data, _offset + rawOffset, _header.Length - rawOffset, 
+                        that._data, that._offset + thatRawOffset, that._header.Length - thatRawOffset);
                 }
             }
 
@@ -270,13 +264,13 @@ namespace Apache.Ignite.Core.Impl.Portable
 
             StringBuilder sb;
 
-            IPortableTypeDescriptor desc = _marsh.GetDescriptor(true, _typeId);
+            IPortableTypeDescriptor desc = _marsh.GetDescriptor(true, _header.TypeId);
 
             IPortableMetadata meta;
 
             try
             {
-                meta = _marsh.GetMetadata(_typeId);
+                meta = _marsh.GetMetadata(_header.TypeId);
             }
             catch (IgniteException)
             {
@@ -284,7 +278,7 @@ namespace Apache.Ignite.Core.Impl.Portable
             }
 
             if (meta == null)
-                sb = new StringBuilder("PortableObject [typeId=").Append(_typeId).Append(", idHash=" + idHash);
+                sb = new StringBuilder("PortableObject [typeId=").Append(_header.TypeId).Append(", idHash=" + idHash);
             else
             {
                 sb = new StringBuilder(meta.TypeName).Append(" [idHash=" + idHash);
@@ -299,7 +293,7 @@ namespace Apache.Ignite.Core.Impl.Portable
                     {
                         sb.Append(", ");
 
-                        int fieldId = PortableUtils.FieldId(_typeId, fieldName, desc.NameConverter, desc.Mapper);
+                        int fieldId = PortableUtils.FieldId(_header.TypeId, fieldName, desc.NameConverter, desc.Mapper);
 
                         int fieldPos;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/b85fa171/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableUtils.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableUtils.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableUtils.cs
index c9d6172..7f9569a 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableUtils.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableUtils.cs
@@ -37,9 +37,6 @@ namespace Apache.Ignite.Core.Impl.Portable
      */
     static class PortableUtils
     {
-        /** Cache empty dictionary. */
-        public static readonly IDictionary<int, int> EmptyFields = new Dictionary<int, int>();
-
         /** Header of NULL object. */
         public const byte HdrNull = 101;
 
@@ -52,21 +49,6 @@ namespace Apache.Ignite.Core.Impl.Portable
         /** Protocol versnion. */
         public const byte ProtoVer = 1;
 
-        /** Full header length. */
-        public const int FullHdrLen = 19;
-
-        /** Offset: hash code. */
-        public const int OffsetTypeId = 3;
-
-        /** Offset: hash code. */
-        public const int OffsetHashCode = 7;
-
-        /** Offset: length. */
-        public const int OffsetLen = 11;
-
-        /** Offset: raw data offset. */
-        public const int OffsetRaw = 15;
-
         /** Type: object. */
         public const byte TypeObject = HdrFull;
 
@@ -253,9 +235,6 @@ namespace Apache.Ignite.Core.Impl.Portable
         /** Indicates object array. */
         public const int ObjTypeId = -1;
 
-        /** Length of tpye ID. */
-        public const int LengthTypeId = 1;
-
         /** Length of array size. */
         public const int LengthArraySize = 4;
 
@@ -1589,56 +1568,6 @@ namespace Apache.Ignite.Core.Impl.Portable
             return id;
         }
 
-        /**
-         * <summary>Get fields map for the given object.</summary>
-         * <param name="stream">Stream.</param>
-         * <param name="typeId">Type ID.</param>
-         * <param name="rawDataOffset">Raw data offset.</param>
-         * <returns>Dictionary with field ID as key and field position as value.</returns>
-         */
-        public static IDictionary<int, int> ObjectFields(IPortableStream stream, int typeId, int rawDataOffset)
-        {
-            int endPos = stream.Position + rawDataOffset - FullHdrLen;
-
-            // First loop detects amount of fields in the object.
-            int retPos = stream.Position;
-            int cnt = 0;
-
-            while (stream.Position < endPos)
-            {
-                cnt++;
-
-                stream.Seek(4, SeekOrigin.Current);
-                int len = stream.ReadInt();
-
-                stream.Seek(stream.Position + len, SeekOrigin.Begin);
-            }
-
-            if (cnt == 0)
-                return EmptyFields;
-
-            stream.Seek(retPos, SeekOrigin.Begin);
-
-            IDictionary<int, int> fields = new Dictionary<int, int>(cnt);
-
-            // Second loop populates fields.
-            while (stream.Position < endPos)
-            {
-                int id = stream.ReadInt();
-                int len = stream.ReadInt();
-
-                if (fields.ContainsKey(id))
-                    throw new PortableException("Object contains duplicate field IDs [typeId=" +
-                        typeId + ", fieldId=" + id + ']');
-
-                fields[id] = stream.Position; // Add field ID and length.
-
-                stream.Seek(stream.Position + len, SeekOrigin.Begin);
-            }
-
-            return fields;
-        }
-
         /// <summary>
         /// Compare contents of two byte array chunks.
         /// </summary>
@@ -1731,13 +1660,11 @@ namespace Apache.Ignite.Core.Impl.Portable
         /// <summary>
         /// Validate protocol version.
         /// </summary>
-        /// <param name="stream">Stream.</param>
-        public static void ValidateProtocolVersion(IPortableStream stream)
+        /// <param name="version">The version.</param>
+        public static void ValidateProtocolVersion(byte version)
         {
-            byte ver = stream.ReadByte();
-
-            if (ver != ProtoVer)
-                throw new PortableException("Unsupported protocol version: " + ver);
+            if (version != ProtoVer)
+                throw new PortableException("Unsupported protocol version: " + version);
         }
 
         /**
@@ -1851,6 +1778,28 @@ namespace Apache.Ignite.Core.Impl.Portable
         }
 
         /// <summary>
+        /// Gets the schema id as a Fnv1 hash.
+        /// </summary>
+        /// <param name="schema">The schema.</param>
+        /// <returns>
+        /// Schema id.
+        /// </returns>
+        public static int GetSchemaId(ResizeableArray<PortableObjectSchemaField> schema)
+        {
+            var hash = Fnv1Hash.Basis;
+
+            if (schema == null || schema.Count == 0)
+                return hash;
+
+            var arr = schema.Array;
+
+            for (int i = 0; i < schema.Count; i++)
+                hash = Fnv1Hash.Update(hash, arr[i].Id);
+
+            return hash;
+        }
+
+        /// <summary>
         /// Reverses the byte order of an unsigned long.
         /// </summary>
         private static ulong ReverseByteOrder(ulong l)

http://git-wip-us.apache.org/repos/asf/ignite/blob/b85fa171/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableWriterImpl.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableWriterImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableWriterImpl.cs
index ab7adaa..e17449d 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableWriterImpl.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableWriterImpl.cs
@@ -21,7 +21,7 @@ namespace Apache.Ignite.Core.Impl.Portable
     using System.Collections;
     using System.Collections.Generic;
     using System.IO;
-
+    using Apache.Ignite.Core.Impl.Common;
     using Apache.Ignite.Core.Impl.Portable.IO;
     using Apache.Ignite.Core.Impl.Portable.Metadata;
     using Apache.Ignite.Core.Impl.Portable.Structure;
@@ -58,15 +58,22 @@ namespace Apache.Ignite.Core.Impl.Portable
         /** Current mapper. */
         private IPortableIdMapper _curMapper;
         
+        /** Current object start position. */
+        private int _curPos;
+
         /** Current raw position. */
-        private long _curRawPos;
+        private int _curRawPos;
 
-        /** Current type structure tracker, */
-        private PortableStructureTracker _curStruct;
-        
         /** Whether we are currently detaching an object. */
         private bool _detaching;
 
+        /** Current type structure tracker, */
+        private PortableStructureTracker _curStruct;
+
+        /** Current schema. */
+        private ResizeableArray<PortableObjectSchemaField> _curSchema;
+
+
         /// <summary>
         /// Gets the marshaller.
         /// </summary>
@@ -84,7 +91,6 @@ namespace Apache.Ignite.Core.Impl.Portable
         {
             WriteFieldId(fieldName, PU.TypeBool);
 
-            _stream.WriteInt(PU.LengthTypeId + 1);
             _stream.WriteByte(PU.TypeBool);
             _stream.WriteBool(val);
         }
@@ -111,7 +117,6 @@ namespace Apache.Ignite.Core.Impl.Portable
                 WriteNullField();
             else
             {
-                _stream.WriteInt(PU.LengthTypeId + PU.LengthArraySize + val.Length);
                 _stream.WriteByte(PU.TypeArrayBool);
                 PU.WriteBooleanArray(val, _stream);
             }
@@ -141,7 +146,6 @@ namespace Apache.Ignite.Core.Impl.Portable
         {
             WriteFieldId(fieldName, PU.TypeBool);
 
-            _stream.WriteInt(PU.LengthTypeId + 1);
             _stream.WriteByte(PU.TypeByte);
             _stream.WriteByte(val);
         }
@@ -168,7 +172,6 @@ namespace Apache.Ignite.Core.Impl.Portable
                 WriteNullField();
             else
             {
-                _stream.WriteInt(PU.LengthTypeId + PU.LengthArraySize + val.Length);
                 _stream.WriteByte(PU.TypeArrayByte);
                 PU.WriteByteArray(val, _stream);
             }
@@ -198,7 +201,6 @@ namespace Apache.Ignite.Core.Impl.Portable
         {
             WriteFieldId(fieldName, PU.TypeShort);
 
-            _stream.WriteInt(PU.LengthTypeId + 2);
             _stream.WriteByte(PU.TypeShort);
             _stream.WriteShort(val);
         }
@@ -225,7 +227,6 @@ namespace Apache.Ignite.Core.Impl.Portable
                 WriteNullField();
             else
             {
-                _stream.WriteInt(PU.LengthTypeId + PU.LengthArraySize + (val.Length << 1));
                 _stream.WriteByte(PU.TypeArrayShort);
                 PU.WriteShortArray(val, _stream);
             }
@@ -255,7 +256,6 @@ namespace Apache.Ignite.Core.Impl.Portable
         {
             WriteFieldId(fieldName, PU.TypeChar);
 
-            _stream.WriteInt(PU.LengthTypeId + 2);
             _stream.WriteByte(PU.TypeChar);
             _stream.WriteChar(val);
         }
@@ -282,7 +282,6 @@ namespace Apache.Ignite.Core.Impl.Portable
                 WriteNullField();
             else
             {
-                _stream.WriteInt(PU.LengthTypeId + PU.LengthArraySize + (val.Length << 1));
                 _stream.WriteByte(PU.TypeArrayChar);
                 PU.WriteCharArray(val, _stream);
             }
@@ -312,7 +311,6 @@ namespace Apache.Ignite.Core.Impl.Portable
         {
             WriteFieldId(fieldName, PU.TypeInt);
 
-            _stream.WriteInt(PU.LengthTypeId + 4);
             _stream.WriteByte(PU.TypeInt);
             _stream.WriteInt(val);
         }
@@ -339,7 +337,6 @@ namespace Apache.Ignite.Core.Impl.Portable
                 WriteNullField();
             else
             {
-                _stream.WriteInt(PU.LengthTypeId + PU.LengthArraySize + (val.Length << 2));
                 _stream.WriteByte(PU.TypeArrayInt);
                 PU.WriteIntArray(val, _stream);
             }
@@ -369,7 +366,6 @@ namespace Apache.Ignite.Core.Impl.Portable
         {
             WriteFieldId(fieldName, PU.TypeLong);
 
-            _stream.WriteInt(PU.LengthTypeId + 8);
             _stream.WriteByte(PU.TypeLong);
             _stream.WriteLong(val);
         }
@@ -396,7 +392,6 @@ namespace Apache.Ignite.Core.Impl.Portable
                 WriteNullField();
             else
             {
-                _stream.WriteInt(PU.LengthTypeId + PU.LengthArraySize + (val.Length << 3));
                 _stream.WriteByte(PU.TypeArrayLong);
                 PU.WriteLongArray(val, _stream);
             }
@@ -426,7 +421,6 @@ namespace Apache.Ignite.Core.Impl.Portable
         {
             WriteFieldId(fieldName, PU.TypeFloat);
 
-            _stream.WriteInt(PU.LengthTypeId + 4);
             _stream.WriteByte(PU.TypeFloat);
             _stream.WriteFloat(val);
         }
@@ -453,7 +447,6 @@ namespace Apache.Ignite.Core.Impl.Portable
                 WriteNullField();
             else
             {
-                _stream.WriteInt(PU.LengthTypeId + PU.LengthArraySize + (val.Length << 2));
                 _stream.WriteByte(PU.TypeArrayFloat);
                 PU.WriteFloatArray(val, _stream);
             }
@@ -483,7 +476,6 @@ namespace Apache.Ignite.Core.Impl.Portable
         {
             WriteFieldId(fieldName, PU.TypeDouble);
 
-            _stream.WriteInt(PU.LengthTypeId + 8);
             _stream.WriteByte(PU.TypeDouble);
             _stream.WriteDouble(val);
         }
@@ -510,7 +502,6 @@ namespace Apache.Ignite.Core.Impl.Portable
                 WriteNullField();
             else
             {
-                _stream.WriteInt(PU.LengthTypeId + PU.LengthArraySize + (val.Length << 3));
                 _stream.WriteByte(PU.TypeArrayDouble);
                 PU.WriteDoubleArray(val, _stream);
             }
@@ -544,12 +535,8 @@ namespace Apache.Ignite.Core.Impl.Portable
                 WriteNullField();
             else
             {
-                int pos = SkipFieldLength();
-
                 _stream.WriteByte(PU.TypeDecimal);
                 PortableUtils.WriteDecimal(val.Value, _stream);
-
-                WriteFieldLength(_stream, pos);
             }
         }
 
@@ -581,12 +568,8 @@ namespace Apache.Ignite.Core.Impl.Portable
                 WriteNullField();
             else
             {
-                int pos = SkipFieldLength();
-
                 _stream.WriteByte(PU.TypeArrayDecimal);
                 PU.WriteDecimalArray(val, _stream);
-
-                WriteFieldLength(_stream, pos);
             }
         }
         
@@ -618,8 +601,6 @@ namespace Apache.Ignite.Core.Impl.Portable
                 WriteNullField();
             else
             {
-                _stream.WriteInt(PU.LengthTypeId + 12);
-
                 _stream.WriteByte(PortableUtils.TypeTimestamp);
                 PortableUtils.WriteTimestamp(val.Value, _stream);
             }
@@ -653,12 +634,8 @@ namespace Apache.Ignite.Core.Impl.Portable
                 WriteNullField();
             else
             {
-                int pos = SkipFieldLength();
-
                 _stream.WriteByte(PortableUtils.TypeArrayTimestamp);
                 PortableUtils.WriteTimestampArray(val, _stream);
-
-                WriteFieldLength(_stream, pos);
             }
         }
 
@@ -690,12 +667,8 @@ namespace Apache.Ignite.Core.Impl.Portable
                 WriteNullField();
             else
             {
-                int pos = SkipFieldLength();
-
                 _stream.WriteByte(PU.TypeString);
                 PU.WriteString(val, _stream);
-
-                WriteFieldLength(_stream, pos);
             }
         }
 
@@ -727,12 +700,8 @@ namespace Apache.Ignite.Core.Impl.Portable
                 WriteNullField();
             else
             {
-                int pos = SkipFieldLength();
-
                 _stream.WriteByte(PU.TypeArrayString);
                 PU.WriteStringArray(val, _stream);
-
-                WriteFieldLength(_stream, pos);
             }
         }
 
@@ -764,8 +733,6 @@ namespace Apache.Ignite.Core.Impl.Portable
                 WriteNullField();
             else
             {
-                _stream.WriteInt(PU.LengthTypeId + 16);
-
                 _stream.WriteByte(PU.TypeGuid);
                 PU.WriteGuid(val.Value, _stream);
             }
@@ -799,12 +766,8 @@ namespace Apache.Ignite.Core.Impl.Portable
                 WriteNullField();
             else
             {
-                int pos = SkipFieldLength();
-
                 _stream.WriteByte(PU.TypeArrayGuid);
                 PU.WriteGuidArray(val, _stream);
-
-                WriteFieldLength(_stream, pos);
             }
         }
 
@@ -833,8 +796,6 @@ namespace Apache.Ignite.Core.Impl.Portable
         {
             WriteFieldId(fieldName, PU.TypeEnum);
 
-            _stream.WriteInt(PU.LengthTypeId + 16);
-
             _stream.WriteByte(PU.TypeEnum);
             PortableUtils.WriteEnum(_stream, (Enum)(object)val);
         }
@@ -864,12 +825,8 @@ namespace Apache.Ignite.Core.Impl.Portable
                 WriteNullField();
             else
             {
-                int pos = SkipFieldLength();
-
                 _stream.WriteByte(PU.TypeArrayEnum);
                 PortableUtils.WriteArray(val, this);
-
-                WriteFieldLength(_stream, pos);
             }
         }
 
@@ -902,13 +859,7 @@ namespace Apache.Ignite.Core.Impl.Portable
             if (val == null)
                 WriteNullField();
             else
-            {
-                int pos = SkipFieldLength();
-
                 Write(val);
-
-                WriteFieldLength(_stream, pos);
-            }
         }
 
         /// <summary>
@@ -935,12 +886,8 @@ namespace Apache.Ignite.Core.Impl.Portable
                 WriteNullField();
             else
             {
-                int pos = SkipFieldLength();
-
                 _stream.WriteByte(PU.TypeArray);
                 PortableUtils.WriteArray(val, this);
-
-                WriteFieldLength(_stream, pos);
             }
         }
 
@@ -981,13 +928,7 @@ namespace Apache.Ignite.Core.Impl.Portable
             if (val == null)
                 WriteNullField();
             else
-            {
-                int pos = SkipFieldLength();
-
                 WriteCollection(val);
-
-                WriteFieldLength(_stream, pos);
-            }
         }
 
         /// <summary>
@@ -1012,13 +953,7 @@ namespace Apache.Ignite.Core.Impl.Portable
             if (val == null)
                 WriteNullField();
             else
-            {
-                int pos = SkipFieldLength();
-
                 WriteDictionary(val);
-
-                WriteFieldLength(_stream, pos);
-            }
         }
 
         /// <summary>
@@ -1036,7 +971,6 @@ namespace Apache.Ignite.Core.Impl.Portable
         /// </summary>
         private void WriteNullField()
         {
-            _stream.WriteInt(1);
             _stream.WriteByte(PU.HdrNull);
         }
 
@@ -1129,44 +1063,51 @@ namespace Apache.Ignite.Core.Impl.Portable
                 if (!(desc.Serializer is IPortableSystemTypeSerializer) && WriteHandle(pos, obj))
                     return;
 
-                // Write header.
-                _stream.WriteByte(PU.HdrFull);
-                _stream.WriteByte(PU.ProtoVer);
-                _stream.WriteBool(desc.UserType);
-                _stream.WriteInt(desc.TypeId);
-                _stream.WriteInt(obj.GetHashCode());
-
-                // Skip length as it is not known in the first place.
-                _stream.Seek(8, SeekOrigin.Current);
+                // Skip header length as not everything is known now
+                _stream.Seek(PortableObjectHeader.Size, SeekOrigin.Current);
 
                 // Preserve old frame.
                 int oldTypeId = _curTypeId;
                 IPortableNameMapper oldConverter = _curConverter;
                 IPortableIdMapper oldMapper = _curMapper;
-                long oldRawPos = _curRawPos;
+                int oldRawPos = _curRawPos;
+                var oldPos = _curPos;
                 
                 var oldStruct = _curStruct;
+                var oldSchema = _curSchema;
 
                 // Push new frame.
                 _curTypeId = desc.TypeId;
                 _curConverter = desc.NameConverter;
                 _curMapper = desc.Mapper;
                 _curRawPos = 0;
+                _curPos = pos;
 
                 _curStruct = new PortableStructureTracker(desc, desc.WriterTypeStructure);
+                _curSchema = null;
 
                 // Write object fields.
                 desc.Serializer.WritePortable(obj, this);
 
-                // Calculate and write length.
-                int len = _stream.Position - pos;
+                // Write schema
+                var hasSchema = _curSchema != null;
+                var schemaOffset = hasSchema ? _stream.Position - pos : PortableObjectHeader.Size;
 
-                _stream.WriteInt(pos + PU.OffsetLen, len);
-                
-                if (_curRawPos != 0)
-                    _stream.WriteInt(pos + PU.OffsetRaw, (int)(_curRawPos - pos));
-                else
-                    _stream.WriteInt(pos + PU.OffsetRaw, len);
+                if (hasSchema)
+                    PortableObjectSchemaField.WriteArray(_curSchema.Array, _stream, _curSchema.Count);
+
+                // Calculate and write header.
+                if (hasSchema && _curRawPos > 0)
+                    _stream.WriteInt(_curRawPos - pos); // raw offset is in the last 4 bytes
+
+                var len = _stream.Position - pos;
+
+                var header = new PortableObjectHeader(desc.UserType, desc.TypeId, obj.GetHashCode(), len,
+                    PU.GetSchemaId(_curSchema), schemaOffset, !hasSchema);
+
+                PortableObjectHeader.Write(header, _stream, pos);
+
+                Stream.Seek(pos + len, SeekOrigin.Begin);  // Seek to the end
 
                 // Apply structure updates if any.
                 _curStruct.UpdateWriterStructure(this);
@@ -1176,8 +1117,10 @@ namespace Apache.Ignite.Core.Impl.Portable
                 _curConverter = oldConverter;
                 _curMapper = oldMapper;
                 _curRawPos = oldRawPos;
+                _curPos = oldPos;
 
                 _curStruct = oldStruct;
+                _curSchema = oldSchema;
             }
             else
             {
@@ -1416,7 +1359,7 @@ namespace Apache.Ignite.Core.Impl.Portable
 
             return true;
         }
-        
+
         /// <summary>
         /// Write field ID.
         /// </summary>
@@ -1427,35 +1370,14 @@ namespace Apache.Ignite.Core.Impl.Portable
             if (_curRawPos != 0)
                 throw new PortableException("Cannot write named fields after raw data is written.");
 
-            _stream.WriteInt(_curStruct.GetFieldId(fieldName, fieldTypeId));
-        }
+            var fieldId = _curStruct.GetFieldId(fieldName, fieldTypeId);
 
-        /// <summary>
-        /// Skip field lenght and return position where it is to be written.
-        /// </summary>
-        /// <returns></returns>
-        private int SkipFieldLength()
-        {
-            int pos = _stream.Position;
+            _curSchema = _curSchema ?? new ResizeableArray<PortableObjectSchemaField>(4);
 
-            _stream.Seek(4, SeekOrigin.Current);
-
-            return pos;
+            _curSchema.Add(new PortableObjectSchemaField(fieldId, _stream.Position - _curPos));
         }
 
         /// <summary>
-        /// Write field length.
-        /// </summary>
-        /// <param name="stream">Stream.</param>
-        /// <param name="pos">Position where length should reside</param>
-        private static void WriteFieldLength(IPortableStream stream, int pos)
-        {
-            // Length is is a difference between current position and previously recorder 
-            // length placeholder position minus 4 bytes for the length itself.
-            stream.WriteInt(pos, stream.Position - pos - 4);
-        }
-        
-        /// <summary>
         /// Saves metadata for this session.
         /// </summary>
         /// <param name="typeId">Type ID.</param>

http://git-wip-us.apache.org/repos/asf/ignite/blob/b85fa171/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortablesImpl.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortablesImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortablesImpl.cs
index 451386b..f48f120 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortablesImpl.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortablesImpl.cs
@@ -164,17 +164,15 @@ namespace Apache.Ignite.Core.Impl.Portable
         /// <returns>Empty portable object.</returns>
         private PortableUserObject PortableFromDescriptor(IPortableTypeDescriptor desc)
         {
-            PortableHeapStream stream = new PortableHeapStream(18);
+            var len = PortableObjectHeader.Size;
 
-            stream.WriteByte(PortableUtils.HdrFull);
-            stream.WriteByte(PortableUtils.ProtoVer);
-            stream.WriteBool(true);
-            stream.WriteInt(desc.TypeId);
-            stream.WriteInt(0); // Hash.
-            stream.WriteInt(PortableUtils.FullHdrLen); // Length.
-            stream.WriteInt(PortableUtils.FullHdrLen); // Raw data offset.
+            var hdr = new PortableObjectHeader(desc.UserType, desc.TypeId, 0, len, 0, len, true);
 
-            return new PortableUserObject(_marsh, stream.InternalArray, 0, desc.TypeId, 0);
+            var stream = new PortableHeapStream(len);
+
+            PortableObjectHeader.Write(hdr, stream, 0);
+
+            return new PortableUserObject(_marsh, stream.InternalArray, 0, hdr);
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/b85fa171/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/Structure/PortableStructureTracker.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/Structure/PortableStructureTracker.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/Structure/PortableStructureTracker.cs
index 8cec87a..4b1165d 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/Structure/PortableStructureTracker.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/Structure/PortableStructureTracker.cs
@@ -54,6 +54,14 @@ namespace Apache.Ignite.Core.Impl.Portable.Structure
         }
 
         /// <summary>
+        /// Gets the current structure action.
+        /// </summary>
+        public int CurStructAction
+        {
+            get { return _curStructAction; }
+        }
+
+        /// <summary>
         /// Gets the field ID.
         /// </summary>
         public int GetFieldId(string fieldName, byte fieldTypeId = 0)

http://git-wip-us.apache.org/repos/asf/ignite/blob/b85fa171/parent/pom.xml
----------------------------------------------------------------------
diff --git a/parent/pom.xml b/parent/pom.xml
index 71a8716..06dfd70 100644
--- a/parent/pom.xml
+++ b/parent/pom.xml
@@ -739,16 +739,11 @@
                                         <exclude>ipc/shmem/igniteshmem/.deps/*</exclude><!--tmp files-->
                                         <exclude>ipc/shmem/igniteshmem/libigniteshmem.la</exclude><!--tmp (not under VCS)-->
                                         <exclude>ipc/shmem/igniteshmem/libigniteshmem_la-org_apache_ignite_internal_util_ipc_shmem_IpcSharedMemoryUtils.lo</exclude><!--tmp (not under VCS)-->
-                                        <!--platform-->
-                                        <exclude>src/main/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj</exclude>
-                                        <exclude>src/main/dotnet/Apache.Ignite.sln</exclude>
-                                        <exclude>src/main/dotnet/Apache.Ignite.sln.DotSettings</exclude>
+                                        <!--platforms-->
                                         <exclude>src/main/java/META-INF/services/org.apache.ignite.internal.processors.platform.PlatformBootstrapFactory</exclude>
                                         <exclude>src/main/resources/META-INF/services/org.apache.ignite.internal.processors.platform.PlatformBootstrapFactory</exclude>
-                                        <exclude>src/test/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj</exclude>
                                         <exclude>src/test/portables/repo/org/apache/ignite/portable/test1/1.1/test1-1.1.jar</exclude>
                                         <exclude>src/test/portables/repo/org/apache/ignite/portable/test2/1.1/test2-1.1.jar</exclude>
-                                        <!--platforms-->
                                         <exclude>**/META-INF/services/*.PlatformBootstrapFactory</exclude>
                                         <exclude>**/Makefile.am</exclude>
                                         <exclude>**/configure.ac</exclude>
@@ -762,6 +757,7 @@
                                         <exclude>**/*.vcxproj</exclude>
                                         <exclude>**/*.vcxprojrel</exclude>
                                         <exclude>**/*.vcxproj.filters</exclude>
+                                        <exclude>**/mkbuild.cmd</exclude>
                                         <exclude>**/module.def</exclude>
                                         <exclude>**/*.fxcop</exclude>
                                         <exclude>**/*.metaproj</exclude>