You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ak...@apache.org on 2015/11/05 04:28:17 UTC

[25/66] [abbrv] ignite git commit: IGNITE-1418: Implemented compact field offsets.

http://git-wip-us.apache.org/repos/asf/ignite/blob/845c4f27/modules/platforms/cpp/core/src/impl/portable/portable_schema.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/src/impl/portable/portable_schema.cpp b/modules/platforms/cpp/core/src/impl/portable/portable_schema.cpp
index 57c23a7..448cf02 100644
--- a/modules/platforms/cpp/core/src/impl/portable/portable_schema.cpp
+++ b/modules/platforms/cpp/core/src/impl/portable/portable_schema.cpp
@@ -15,6 +15,8 @@
 * limitations under the License.
 */
 
+#include <cassert>
+
 #include "ignite/impl/portable/portable_schema.h"
 #include "ignite/impl/portable/portable_writer_impl.h"
 
@@ -66,10 +68,43 @@ namespace ignite
 
             void PortableSchema::Write(interop::InteropOutputStream& out) const
             {
-                for (FieldContainer::const_iterator i = fieldsInfo->begin(); i != fieldsInfo->end(); ++i)
+                switch (GetType())
                 {
-                    out.WriteInt32(i->id);
-                    out.WriteInt32(i->offset);
+                    case OFFSET_TYPE_1_BYTE:
+                    {
+                        for (FieldContainer::const_iterator i = fieldsInfo->begin(); i != fieldsInfo->end(); ++i)
+                        {
+                            out.WriteInt32(i->id);
+                            out.WriteInt8(static_cast<int8_t>(i->offset));
+                        }
+                        break;
+                    }
+
+                    case OFFSET_TYPE_2_BYTE:
+                    {
+                        for (FieldContainer::const_iterator i = fieldsInfo->begin(); i != fieldsInfo->end(); ++i)
+                        {
+                            out.WriteInt32(i->id);
+                            out.WriteInt16(static_cast<int16_t>(i->offset));
+                        }
+                        break;
+                    }
+
+                    case OFFSET_TYPE_4_BYTE:
+                    {
+                        for (FieldContainer::const_iterator i = fieldsInfo->begin(); i != fieldsInfo->end(); ++i)
+                        {
+                            out.WriteInt32(i->id);
+                            out.WriteInt32(i->offset);
+                        }
+                        break;
+                    }
+
+                    default:
+                    {
+                        assert(false);
+                        break;
+                    }
                 }
             }
 
@@ -83,6 +118,18 @@ namespace ignite
                 id = 0;
                 fieldsInfo->clear();
             }
+
+            PortableOffsetType PortableSchema::GetType() const
+            {
+                int32_t maxOffset = fieldsInfo->back().offset;
+
+                if (maxOffset < 0x100)
+                    return OFFSET_TYPE_1_BYTE;
+                else if (maxOffset < 0x10000)
+                    return OFFSET_TYPE_2_BYTE;
+
+                return OFFSET_TYPE_4_BYTE;
+            }
         }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/845c4f27/modules/platforms/cpp/core/src/impl/portable/portable_writer_impl.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/src/impl/portable/portable_writer_impl.cpp b/modules/platforms/cpp/core/src/impl/portable/portable_writer_impl.cpp
index 4b65b4b..2dac125 100644
--- a/modules/platforms/cpp/core/src/impl/portable/portable_writer_impl.cpp
+++ b/modules/platforms/cpp/core/src/impl/portable/portable_writer_impl.cpp
@@ -567,13 +567,12 @@ namespace ignite
             {
                 int32_t lenWithoutSchema = stream->Position() - start;
 
+                int32_t nonRawLen = rawPos == -1 ? lenWithoutSchema : rawPos - start;
+                
                 if (schema.Empty())
                 {
-                    InteropStreamPositionGuard<InteropOutputStream> guard(*stream);
-
-                    stream->Position(start + IGNITE_OFFSET_FLAGS);
-                    stream->WriteInt16(IGNITE_PORTABLE_FLAG_USER_OBJECT | IGNITE_PORTABLE_FLAG_RAW_ONLY);
-
+                    stream->WriteInt16(start + IGNITE_OFFSET_FLAGS, IGNITE_PORTABLE_FLAG_USER_OBJECT | 
+                                                                    IGNITE_PORTABLE_FLAG_RAW_ONLY);
                     stream->WriteInt32(start + IGNITE_OFFSET_LEN, lenWithoutSchema);
                     stream->WriteInt32(start + IGNITE_OFFSET_SCHEMA_ID, 0);
                     stream->WriteInt32(start + IGNITE_OFFSET_SCHEMA_OR_RAW_OFF, GetRawPosition() - start);
@@ -581,6 +580,7 @@ namespace ignite
                 else
                 {
                     int32_t schemaId = schema.GetId();
+                    PortableOffsetType schemaType = schema.GetType();
 
                     WriteAndClearSchema();
 
@@ -589,6 +589,17 @@ namespace ignite
 
                     int32_t length = stream->Position() - start;
 
+                    if (schemaType == OFFSET_TYPE_1_BYTE)
+                    {
+                        stream->WriteInt16(start + IGNITE_OFFSET_FLAGS, 
+                            IGNITE_PORTABLE_FLAG_USER_OBJECT | IGNITE_PORTABLE_FLAG_OFFSET_1_BYTE);
+                    }
+                    else if (schemaType == OFFSET_TYPE_2_BYTE)
+                    {
+                        stream->WriteInt16(start + IGNITE_OFFSET_FLAGS, 
+                            IGNITE_PORTABLE_FLAG_USER_OBJECT | IGNITE_PORTABLE_FLAG_OFFSET_2_BYTE);
+                    }
+
                     stream->WriteInt32(start + IGNITE_OFFSET_LEN, length);
                     stream->WriteInt32(start + IGNITE_OFFSET_SCHEMA_ID, schemaId);
                     stream->WriteInt32(start + IGNITE_OFFSET_SCHEMA_OR_RAW_OFF, lenWithoutSchema);

http://git-wip-us.apache.org/repos/asf/ignite/blob/845c4f27/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Portable/PortableSelfTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Portable/PortableSelfTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Portable/PortableSelfTest.cs
index 08e0b31..2313f92 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Portable/PortableSelfTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Portable/PortableSelfTest.cs
@@ -1225,6 +1225,38 @@ namespace Apache.Ignite.Core.Tests.Portable
             Assert.AreEqual(nDateArr, obj2.NDateArr);
         }
 
+        /// <summary>
+        /// Writes objects of various sizes to test schema compaction 
+        /// (where field offsets can be stored as 1, 2 or 4 bytes).
+        /// </summary>
+        [Test]
+        public void TestCompactSchema()
+        {
+            var marsh = new PortableMarshaller(new PortableConfiguration
+            {
+                TypeConfigurations = new List<PortableTypeConfiguration>
+                {
+                    new PortableTypeConfiguration(typeof (SpecialArray)),
+                    new PortableTypeConfiguration(typeof (SpecialArrayMarshalAware))
+                }
+            });
+
+            var dt = new SpecialArrayMarshalAware();
+
+            foreach (var i in new[] {1, 5, 10, 13, 14, 15, 100, 200, 1000, 5000, 15000, 30000})
+            {
+                dt.NGuidArr = Enumerable.Range(1, i).Select(x => (Guid?) Guid.NewGuid()).ToArray();
+                dt.NDateArr = Enumerable.Range(1, i).Select(x => (DateTime?) DateTime.Now.AddDays(x)).ToArray();
+
+                var bytes = marsh.Marshal(dt);
+
+                var res = marsh.Unmarshal<SpecialArrayMarshalAware>(bytes);
+
+                CollectionAssert.AreEquivalent(dt.NGuidArr, res.NGuidArr);
+                CollectionAssert.AreEquivalent(dt.NDateArr, res.NDateArr);
+            }
+        }
+
         private static void CheckKeepSerialized(PortableConfiguration cfg, bool expKeep)
         {
             if (cfg.TypeConfigurations == null)

http://git-wip-us.apache.org/repos/asf/ignite/blob/845c4f27/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/ResizeableArray.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/ResizeableArray.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/ResizeableArray.cs
index cb432e5..82a8eee 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/ResizeableArray.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/ResizeableArray.cs
@@ -1,4 +1,4 @@
-/*
+/*
  * 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.
@@ -27,9 +27,6 @@ namespace Apache.Ignite.Core.Impl.Common
         /** Array. */
         private T[] _arr;
 
-        /** Items count. */
-        private int _count;
-
         /// <summary>
         /// Constructor.
         /// </summary>
@@ -50,10 +47,7 @@ namespace Apache.Ignite.Core.Impl.Common
         /// <summary>
         /// Count.
         /// </summary>
-        public int Count
-        {
-            get { return _count; }
-        }
+        public int Count { get; private set; }
 
         /// <summary>
         /// Add element.
@@ -61,10 +55,10 @@ namespace Apache.Ignite.Core.Impl.Common
         /// <param name="element">Element.</param>
         public void Add(T element)
         {
-            if (_count == _arr.Length)
+            if (Count == _arr.Length)
                 System.Array.Resize(ref _arr, _arr.Length*2);
 
-            _arr[_count++] = element;
+            _arr[Count++] = element;
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/845c4f27/modules/platforms/dotnet/Apache.Ignite.Core/Impl/ExceptionUtils.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/ExceptionUtils.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/ExceptionUtils.cs
index ea109ba..c12a651 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/ExceptionUtils.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/ExceptionUtils.cs
@@ -21,7 +21,6 @@ namespace Apache.Ignite.Core.Impl
     using System.Collections.Generic;
     using System.Diagnostics;
     using System.Diagnostics.CodeAnalysis;
-    using System.Runtime.InteropServices;
     using System.Security;
     using System.Threading;
     using Apache.Ignite.Core.Cache;
@@ -35,7 +34,6 @@ namespace Apache.Ignite.Core.Impl
     /// <summary>
     /// Managed environment. Acts as a gateway for native code.
     /// </summary>
-    [StructLayout(LayoutKind.Sequential)]
     internal static class ExceptionUtils
     {
         /** NoClassDefFoundError fully-qualified class name which is important during startup phase. */

http://git-wip-us.apache.org/repos/asf/ignite/blob/845c4f27/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableBuilderImpl.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableBuilderImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableBuilderImpl.cs
index 08a1d00..c299039 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableBuilderImpl.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableBuilderImpl.cs
@@ -569,7 +569,7 @@ namespace Apache.Ignite.Core.Impl.Portable
         /// <param name="hash">New hash.</param>
         /// <param name="vals">Values to be replaced.</param>
         /// <returns>Mutated object.</returns>
-        private unsafe void Mutate0(Context ctx, PortableHeapStream inStream, IPortableStream outStream,
+        private void Mutate0(Context ctx, PortableHeapStream inStream, IPortableStream outStream,
             bool changeHash, int hash, IDictionary<int, PortableBuilderField> vals)
         {
             int inStartPos = inStream.Position;
@@ -694,12 +694,14 @@ namespace Apache.Ignite.Core.Impl.Portable
 
                         // Write schema
                         int outSchemaOff = outRawOff;
+                        short flags = 0;
 
                         if (outSchema != null)
                         {
                             outSchemaOff = outStream.Position - outStartPos;
 
-                            PortableObjectSchemaField.WriteArray(outSchema.Array, outStream, outSchema.Count);
+                            flags = PortableObjectHeader.WriteSchema(outSchema.Array, outStream, outSchema.Count,
+                                outStream.Position - outStartPos);
 
                             if (inRawLen > 0)
                                 outStream.WriteInt(outRawOff);
@@ -712,7 +714,7 @@ namespace Apache.Ignite.Core.Impl.Portable
                         var outHash = changeHash ? hash : inHeader.HashCode;
 
                         var outHeader = new PortableObjectHeader(inHeader.IsUserType, inHeader.TypeId, outHash, 
-                            outLen, outSchemaId, outSchemaOff, outSchema == null);
+                            outLen, outSchemaId, outSchemaOff, outSchema == null, flags);
 
                         PortableObjectHeader.Write(outHeader, outStream, outStartPos);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/845c4f27/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableObjectHeader.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableObjectHeader.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableObjectHeader.cs
index b3768a0..50adc02 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableObjectHeader.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableObjectHeader.cs
@@ -27,17 +27,23 @@ namespace Apache.Ignite.Core.Impl.Portable
     /// <summary>
     /// Portable object header structure.
     /// </summary>
-    [StructLayout(LayoutKind.Sequential)]
+    [StructLayout(LayoutKind.Sequential, Pack = 0)]
     internal struct PortableObjectHeader : IEquatable<PortableObjectHeader>
     {
-        /** Size, equals to sizeof(PortableObjectHeader) */
+        /** Size, equals to sizeof(PortableObjectHeader). */
         public const int Size = 24;
 
-        /** User type flag */
-        private const int FlagUserType = 0x1;
+        /** User type flag. */
+        public const short FlagUserType = 0x1;
 
-        /** Raw only flag */
-        private const int FlagRawOnly = 0x2;
+        /** Raw only flag. */
+        public const short FlagRawOnly = 0x2;
+
+        /** Byte-sized field offsets flag. */
+        public const short FlagByteOffsets = 0x4;
+
+        /** Short-sized field offsets flag. */
+        public const short FlagShortOffsets = 0x8;
 
         /** Actual header layout */
         public readonly byte Header;        // Header code, always 103 (HdrFull)
@@ -50,7 +56,7 @@ namespace Apache.Ignite.Core.Impl.Portable
         public readonly int SchemaOffset;   // Schema offset, or raw offset when RawOnly flag is set.
 
         /// <summary>
-        /// Initializes a new instance of the <see cref="PortableObjectHeader"/> struct.
+        /// Initializes a new instance of the <see cref="PortableObjectHeader" /> struct.
         /// </summary>
         /// <param name="userType">User type flag.</param>
         /// <param name="typeId">Type ID.</param>
@@ -59,18 +65,23 @@ namespace Apache.Ignite.Core.Impl.Portable
         /// <param name="schemaId">Schema ID.</param>
         /// <param name="schemaOffset">Schema offset.</param>
         /// <param name="rawOnly">Raw flag.</param>
-        public PortableObjectHeader(bool userType, int typeId, int hashCode, int length, int schemaId, int schemaOffset, bool rawOnly)
+        /// <param name="flags">The flags.</param>
+        public PortableObjectHeader(bool userType, int typeId, int hashCode, int length, int schemaId, int schemaOffset, 
+            bool rawOnly, short flags)
         {
             Header = PortableUtils.HdrFull;
             Version = PortableUtils.ProtoVer;
 
             Debug.Assert(schemaOffset <= length);
             Debug.Assert(schemaOffset >= Size);
-            
-            Flags = (short) (userType ? FlagUserType : 0);
+
+            if (userType)
+                flags |= FlagUserType;
 
             if (rawOnly)
-                Flags = (short) (Flags | FlagRawOnly);
+                flags |= FlagRawOnly;
+
+            Flags = flags;
 
             TypeId = typeId;
             HashCode = hashCode;
@@ -134,49 +145,50 @@ namespace Apache.Ignite.Core.Impl.Portable
         {
             get
             {
-                // Odd amount of records in schema => raw offset is the very last 4 bytes in object.
-                return !IsRawOnly && (((Length - SchemaOffset) >> 2) & 0x1) != 0x0;
+                // Remainder => raw offset is the very last 4 bytes in object.
+                return !IsRawOnly && ((Length - SchemaOffset) % SchemaFieldSize) == 4;
             }
         }
 
         /// <summary>
-        /// Gets the schema field count.
+        /// Gets the size of the schema field offset (1, 2 or 4 bytes).
         /// </summary>
-        public int SchemaFieldCount
+        public int SchemaFieldOffsetSize
         {
             get
             {
-                if (IsRawOnly)
-                    return 0;
-
-                var schemaSize = Length - SchemaOffset;
+                if ((Flags & FlagByteOffsets) == FlagByteOffsets)
+                    return 1;
 
-                if (HasRawOffset)
-                    schemaSize -= 4;
+                if ((Flags & FlagShortOffsets) == FlagShortOffsets)
+                    return 2;
 
-                return schemaSize >> 3;  // 8 == PortableObjectSchemaField.Size
+                return 4;
             }
         }
 
         /// <summary>
-        /// Gets the schema end.
+        /// Gets the size of the schema field.
         /// </summary>
-        public int GetSchemaEnd(int position)
+        public int SchemaFieldSize
         {
-            var res = position + Length;
-
-            if (HasRawOffset)
-                res -= 4;
-
-            return res;
+            get { return SchemaFieldOffsetSize + 4; }
         }
 
         /// <summary>
-        /// Gets the schema start.
+        /// Gets the schema field count.
         /// </summary>
-        public int GetSchemaStart(int position)
+        public int SchemaFieldCount
         {
-            return IsRawOnly ? GetSchemaEnd(position) : position + SchemaOffset;
+            get
+            {
+                if (IsRawOnly)
+                    return 0;
+
+                var schemaSize = Length - SchemaOffset;
+
+                return schemaSize / SchemaFieldSize;
+            }
         }
 
         /// <summary>
@@ -214,10 +226,25 @@ namespace Apache.Ignite.Core.Impl.Portable
 
             stream.Seek(position + SchemaOffset, SeekOrigin.Begin);
 
-            var schema = new Dictionary<int, int>(schemaSize >> 3);
+            var schema = new Dictionary<int, int>(schemaSize);
+
+            var offsetSize = SchemaFieldOffsetSize;
 
-            for (var i = 0; i < schemaSize; i++)
-                schema.Add(stream.ReadInt(), stream.ReadInt());
+            if (offsetSize == 1)
+            {
+                for (var i = 0; i < schemaSize; i++)
+                    schema.Add(stream.ReadInt(), stream.ReadByte());
+            }
+            else if (offsetSize == 2)
+            {
+                for (var i = 0; i < schemaSize; i++)
+                    schema.Add(stream.ReadInt(), stream.ReadShort());
+            }
+            else
+            {
+                for (var i = 0; i < schemaSize; i++)
+                    schema.Add(stream.ReadInt(), stream.ReadInt());
+            }
 
             return schema;
         }
@@ -239,7 +266,98 @@ namespace Apache.Ignite.Core.Impl.Portable
 
             stream.Seek(position + SchemaOffset, SeekOrigin.Begin);
 
-            return PortableObjectSchemaField.ReadArray(stream, schemaSize);
+            var schema = new PortableObjectSchemaField[schemaSize];
+
+            var offsetSize = SchemaFieldOffsetSize;
+
+            if (offsetSize == 1)
+            {
+                for (var i = 0; i < schemaSize; i++)
+                    schema[i] = new PortableObjectSchemaField(stream.ReadInt(), stream.ReadByte());
+            }
+            else if (offsetSize == 2)
+            {
+                for (var i = 0; i < schemaSize; i++)
+                    schema[i] = new PortableObjectSchemaField(stream.ReadInt(), stream.ReadShort());
+            }
+            else
+            {
+                for (var i = 0; i < schemaSize; i++)
+                    schema[i] = new PortableObjectSchemaField(stream.ReadInt(), stream.ReadInt());
+            }
+
+            return schema;
+        }
+
+        /// <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>
+        /// <param name="maxOffset">The maximum field offset to determine 
+        /// whether 1, 2 or 4 bytes are needed for offsets.</param>
+        /// <returns>
+        /// Flags according to offset sizes: <see cref="PortableObjectHeader.FlagByteOffsets"/>, 
+        /// <see cref="PortableObjectHeader.FlagShortOffsets"/>, or 0.
+        /// </returns>
+        public static unsafe short WriteSchema(PortableObjectSchemaField[] fields, IPortableStream stream, int count,
+            int maxOffset)
+        {
+            Debug.Assert(fields != null);
+            Debug.Assert(stream != null);
+            Debug.Assert(count > 0);
+
+            unchecked
+            {
+                if (maxOffset <= byte.MaxValue)
+                {
+                    for (int i = 0; i < count; i++)
+                    {
+                        var field = fields[i];
+
+                        stream.WriteInt(field.Id);
+                        stream.WriteByte((byte)field.Offset);
+                    }
+
+                    return FlagByteOffsets;
+                }
+
+                if (maxOffset <= ushort.MaxValue)
+                {
+                    for (int i = 0; i < count; i++)
+                    {
+                        var field = fields[i];
+
+                        stream.WriteInt(field.Id);
+
+                        stream.WriteShort((short)field.Offset);
+                    }
+
+                    return FlagShortOffsets;
+                }
+
+                if (BitConverter.IsLittleEndian)
+                {
+                    fixed (PortableObjectSchemaField* ptr = &fields[0])
+                    {
+                        stream.Write((byte*)ptr, count / PortableObjectSchemaField.Size);
+                    }
+                }
+                else
+                {
+                    for (int i = 0; i < count; i++)
+                    {
+                        var field = fields[i];
+
+                        stream.WriteInt(field.Id);
+                        stream.WriteInt(field.Offset);
+                    }
+                }
+
+                return 0;
+            }
+
         }
 
         /// <summary>
@@ -284,6 +402,10 @@ namespace Apache.Ignite.Core.Impl.Portable
                 Debug.Assert(hdr.SchemaOffset <= hdr.Length);
                 Debug.Assert(hdr.SchemaOffset >= Size);
 
+                // Only one of the flags can be set
+                var f = hdr.Flags;
+                Debug.Assert((f & (FlagShortOffsets | FlagByteOffsets)) != (FlagShortOffsets | FlagByteOffsets));
+
                 return hdr;
             }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/845c4f27/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
index 7d3663c..51ae34e 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableObjectSchema.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableObjectSchema.cs
@@ -20,7 +20,7 @@ namespace Apache.Ignite.Core.Impl.Portable
     using System.Collections.Generic;
 
     /// <summary>
-    /// Holds and manages portable object schema for a specific type.
+    /// Holds and manages portable object schemas for a specific type.
     /// </summary>
     internal class PortableObjectSchema
     {
@@ -44,7 +44,7 @@ namespace Apache.Ignite.Core.Impl.Portable
         /// </summary>
         /// <param name="id">Schema id.</param>
         /// <returns>Schema or null.</returns>
-        public int[] GetSchema(int id)
+        public int[] Get(int id)
         {
             if (_schemaId1 == id)
                 return _schema1;
@@ -65,7 +65,7 @@ namespace Apache.Ignite.Core.Impl.Portable
         /// </summary>
         /// <param name="id">Schema id.</param>
         /// <param name="schema">Schema.</param>
-        public void AddSchema(int id, int[] schema)
+        public void Add(int id, int[] schema)
         {
             lock (this)
             {

http://git-wip-us.apache.org/repos/asf/ignite/blob/845c4f27/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
index 48fd9c1..bc18191 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableObjectSchemaField.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableObjectSchemaField.cs
@@ -17,15 +17,12 @@
 
 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)]
+    [StructLayout(LayoutKind.Sequential, Pack = 0)]
     internal struct PortableObjectSchemaField
     {
         /* Field ID */
@@ -35,7 +32,7 @@ namespace Apache.Ignite.Core.Impl.Portable
         public readonly int Offset;
 
         /** Size, equals to sizeof(PortableObjectSchemaField) */
-        private const int Size = 8;
+        public const int Size = 8;
 
         /// <summary>
         /// Initializes a new instance of the <see cref="PortableObjectSchemaField"/> struct.
@@ -47,67 +44,5 @@ namespace Apache.Ignite.Core.Impl.Portable
             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/845c4f27/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 48ea799..2b7ddb8 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableReaderImpl.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableReaderImpl.cs
@@ -45,15 +45,9 @@ namespace Apache.Ignite.Core.Impl.Portable
         /** Handles. */
         private PortableReaderHandleDictionary _hnds;
 
-        /** Current type ID. */
-        private int _curTypeId;
-
         /** Current position. */
         private int _curPos;
 
-        /** Current raw data offset. */
-        private int _curRawOffset;
-
         /** Current raw flag. */
         private bool _curRaw;
 
@@ -66,14 +60,14 @@ namespace Apache.Ignite.Core.Impl.Portable
         /** Current type structure tracker. */
         private PortableStructureTracker _curStruct;
 
-        /** */
-        private int _curFooterStart;
+        /** Current schema. */
+        private int[] _curSchema;
 
-        /** */
-        private int _curFooterEnd;
+        /** Current schema with positions. */
+        private Dictionary<int, int> _curSchemaMap;
 
-        /** */
-        private int[] _curSchema;
+        /** Current header. */
+        private PortableObjectHeader _curHdr;
 
         /// <summary>
         /// Constructor.
@@ -432,9 +426,7 @@ namespace Apache.Ignite.Core.Impl.Portable
             if (_curRaw)
                 throw new PortableException("Cannot read named fields after raw data is read.");
 
-            int fieldId = _curStruct.GetFieldId(fieldName);
-
-            if (SeekField(fieldId))
+            if (SeekField(fieldName))
                 return Deserialize<T>();
 
             return default(T);
@@ -669,31 +661,26 @@ namespace Apache.Ignite.Core.Impl.Portable
                                                     desc.TypeId + ", typeName=" + desc.TypeName + ']');
 
                     // Preserve old frame.
-                    int oldTypeId = _curTypeId;
+                    var oldHdr = _curHdr;
                     int oldPos = _curPos;
-                    int oldRawOffset = _curRawOffset;
                     var oldStruct = _curStruct;
                     bool oldRaw = _curRaw;
-                    var oldFooterStart = _curFooterStart;
-                    var oldFooterEnd = _curFooterEnd;
                     var oldSchema = _curSchema;
+                    var oldSchemaMap = _curSchemaMap;
 
                     // Set new frame.
-                    _curTypeId = hdr.TypeId;
+                    _curHdr = hdr;
                     _curPos = pos;
-                    _curFooterEnd = hdr.GetSchemaEnd(pos);
-                    _curFooterStart = hdr.GetSchemaStart(pos);
                     
-                    _curSchema = desc.Schema.GetSchema(hdr.SchemaId);
+                    _curSchema = desc.Schema.Get(hdr.SchemaId);
 
                     if (_curSchema == null)
                     {
                         _curSchema = ReadSchema();
 
-                        desc.Schema.AddSchema(hdr.SchemaId, _curSchema);
+                        desc.Schema.Add(hdr.SchemaId, _curSchema);
                     }
 
-                    _curRawOffset = hdr.GetRawOffset(Stream, pos);
                     _curStruct = new PortableStructureTracker(desc, desc.ReaderTypeStructure);
                     _curRaw = false;
 
@@ -727,14 +714,12 @@ namespace Apache.Ignite.Core.Impl.Portable
                     _curStruct.UpdateReaderStructure();
 
                     // Restore old frame.
-                    _curTypeId = oldTypeId;
+                    _curHdr = oldHdr;
                     _curPos = oldPos;
-                    _curRawOffset = oldRawOffset;
                     _curStruct = oldStruct;
                     _curRaw = oldRaw;
-                    _curFooterStart = oldFooterStart;
-                    _curFooterEnd = oldFooterEnd;
                     _curSchema = oldSchema;
+                    _curSchemaMap = oldSchemaMap;
 
                     // Process wrappers. We could introduce a common interface, but for only 2 if-else is faster.
                     var wrappedSerializable = obj as SerializableObjectHolder;
@@ -762,21 +747,22 @@ namespace Apache.Ignite.Core.Impl.Portable
         /// </summary>
         private int[] ReadSchema()
         {
-            Stream.Seek(_curFooterStart, SeekOrigin.Begin);
-            
-            var count = (_curFooterEnd - _curFooterStart) >> 3;
-            
+            Stream.Seek(_curPos + _curHdr.SchemaOffset, SeekOrigin.Begin);
+
+            var count = _curHdr.SchemaFieldCount;
+
+            var offsetSize = _curHdr.SchemaFieldOffsetSize;
+
             var res = new int[count];
 
             for (int i = 0; i < count; i++)
             {
                 res[i] = Stream.ReadInt();
-                Stream.Seek(4, SeekOrigin.Current);
+                Stream.Seek(offsetSize, SeekOrigin.Current);
             }
 
             return res;
         }
-
         /// <summary>
         /// Reads the handle object.
         /// </summary>
@@ -846,38 +832,11 @@ namespace Apache.Ignite.Core.Impl.Portable
             {
                 _curRaw = true;
 
-                Stream.Seek(_curPos + _curRawOffset, SeekOrigin.Begin);
+                Stream.Seek(_curPos + _curHdr.GetRawOffset(Stream, _curPos), SeekOrigin.Begin);
             }
         }
 
         /// <summary>
-        /// Seek field with the given ID in the current object.
-        /// </summary>
-        /// <param name="fieldId">Field ID.</param>
-        /// <returns>True in case the field was found and position adjusted, false otherwise.</returns>
-        private bool SeekField(int fieldId)
-        {
-            Stream.Seek(_curFooterStart, SeekOrigin.Begin);
-
-            while (Stream.Position < _curFooterEnd)
-            {
-                var id = Stream.ReadInt();
-
-                if (id == fieldId)
-                {
-                    var fieldOffset = Stream.ReadInt();
-
-                    Stream.Seek(_curPos + fieldOffset, SeekOrigin.Begin);
-                    return true;
-                }
-
-                Stream.Seek(4, SeekOrigin.Current);
-            }
-
-            return false;
-        }
-
-        /// <summary>
         /// Determines whether header at current position is HDR_NULL.
         /// </summary>
         private bool IsNotNullHeader(byte expHdr)
@@ -899,23 +858,43 @@ namespace Apache.Ignite.Core.Impl.Portable
         /// </summary>
         private bool SeekField(string fieldName, byte expHdr)
         {
+            if (!SeekField(fieldName)) 
+                return false;
+
+            // Expected read order, no need to seek.
+            return IsNotNullHeader(expHdr);
+        }
+
+        /// <summary>
+        /// Seeks the field by name.
+        /// </summary>
+        private bool SeekField(string fieldName)
+        {
             if (_curRaw)
                 throw new PortableException("Cannot read named fields after raw data is read.");
 
+            if (_curHdr.IsRawOnly)
+                return false;
+
             var actionId = _curStruct.CurStructAction;
 
             var fieldId = _curStruct.GetFieldId(fieldName);
 
             if (_curSchema == null || actionId >= _curSchema.Length || fieldId != _curSchema[actionId])
             {
-                _curSchema = null;   // read order is different, ignore schema for future reads
+                _curSchema = null; // read order is different, ignore schema for future reads
+
+                _curSchemaMap = _curSchemaMap ?? _curHdr.ReadSchemaAsDictionary(Stream, _curPos);
+
+                int pos;
 
-                if (!SeekField(fieldId))
+                if (!_curSchemaMap.TryGetValue(fieldId, out pos))
                     return false;
+
+                Stream.Seek(pos, SeekOrigin.Begin);
             }
 
-            // Expected read order, no need to seek.
-            return IsNotNullHeader(expHdr);
+            return true;
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/845c4f27/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 a0657b2..79c4858 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableUtils.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableUtils.cs
@@ -1804,7 +1804,7 @@ namespace Apache.Ignite.Core.Impl.Portable
         /// <summary>
         /// Struct with .Net-style Guid memory layout.
         /// </summary>
-        [StructLayout(LayoutKind.Sequential)]
+        [StructLayout(LayoutKind.Sequential, Pack = 0)]
         private struct GuidAccessor
         {
             public readonly ulong ABC;
@@ -1828,7 +1828,7 @@ namespace Apache.Ignite.Core.Impl.Portable
         /// <summary>
         /// Struct with Java-style Guid memory layout.
         /// </summary>
-        [StructLayout(LayoutKind.Sequential)]
+        [StructLayout(LayoutKind.Sequential, Pack = 0)]
         private struct JavaGuid
         {
             public readonly ulong CBA;

http://git-wip-us.apache.org/repos/asf/ignite/blob/845c4f27/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 e17449d..66caca3 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableWriterImpl.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableWriterImpl.cs
@@ -73,7 +73,6 @@ namespace Apache.Ignite.Core.Impl.Portable
         /** Current schema. */
         private ResizeableArray<PortableObjectSchemaField> _curSchema;
 
-
         /// <summary>
         /// Gets the marshaller.
         /// </summary>
@@ -1092,9 +1091,11 @@ namespace Apache.Ignite.Core.Impl.Portable
                 // Write schema
                 var hasSchema = _curSchema != null;
                 var schemaOffset = hasSchema ? _stream.Position - pos : PortableObjectHeader.Size;
+                short flags = 0;
 
                 if (hasSchema)
-                    PortableObjectSchemaField.WriteArray(_curSchema.Array, _stream, _curSchema.Count);
+                    flags = PortableObjectHeader.WriteSchema(_curSchema.Array, _stream, _curSchema.Count,
+                        _curSchema.Array[_curSchema.Count - 1].Offset);
 
                 // Calculate and write header.
                 if (hasSchema && _curRawPos > 0)
@@ -1103,7 +1104,7 @@ namespace Apache.Ignite.Core.Impl.Portable
                 var len = _stream.Position - pos;
 
                 var header = new PortableObjectHeader(desc.UserType, desc.TypeId, obj.GetHashCode(), len,
-                    PU.GetSchemaId(_curSchema), schemaOffset, !hasSchema);
+                    PU.GetSchemaId(_curSchema), schemaOffset, !hasSchema, flags);
 
                 PortableObjectHeader.Write(header, _stream, pos);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/845c4f27/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 f48f120..e72ffac 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortablesImpl.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortablesImpl.cs
@@ -166,7 +166,7 @@ namespace Apache.Ignite.Core.Impl.Portable
         {
             var len = PortableObjectHeader.Size;
 
-            var hdr = new PortableObjectHeader(desc.UserType, desc.TypeId, 0, len, 0, len, true);
+            var hdr = new PortableObjectHeader(desc.UserType, desc.TypeId, 0, len, 0, len, true, 0);
 
             var stream = new PortableHeapStream(len);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/845c4f27/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbackHandlers.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbackHandlers.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbackHandlers.cs
index 07cf309..8147e9d 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbackHandlers.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbackHandlers.cs
@@ -22,7 +22,7 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
     /// <summary>
     /// Unmanaged callback handler function pointers.
     /// </summary>
-    [StructLayout(LayoutKind.Sequential)]
+    [StructLayout(LayoutKind.Sequential, Pack = 0)]
     internal unsafe struct UnmanagedCallbackHandlers
     {
         internal void* target;