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/11/04 11:28:11 UTC

[08/16] ignite git commit: IGNITE-1835: .Net: Use thread-local pool to store schemas in PortableWriter. About ~5 write performance gain is achieved.

IGNITE-1835: .Net: Use thread-local pool to store schemas in PortableWriter. About ~5 write performance gain is achieved.


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

Branch: refs/heads/ignite-950-new
Commit: e4b128e39cb88435c564bcc8ccc2bc913d55c2ca
Parents: a88ada4
Author: Pavel Tupitsyn <pt...@gridgain.com>
Authored: Wed Nov 4 11:50:24 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Wed Nov 4 11:50:24 2015 +0300

----------------------------------------------------------------------
 .../internal/portable/PortableWriterExImpl.java |   4 +-
 .../Apache.Ignite.Core.csproj                   |   1 +
 .../Apache.Ignite.Core/Impl/Common/Fnv1Hash.cs  |  21 +--
 .../Impl/Portable/PortableBuilderImpl.cs        | 133 +++++++++----------
 .../Impl/Portable/PortableObjectHeader.cs       |  28 ++--
 .../Impl/Portable/PortableObjectSchemaHolder.cs | 108 +++++++++++++++
 .../Impl/Portable/PortableUtils.cs              |  22 ---
 .../Impl/Portable/PortableWriterImpl.cs         |  55 ++++----
 8 files changed, 234 insertions(+), 138 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e4b128e3/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableWriterExImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableWriterExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableWriterExImpl.java
index ff85ecd..542c897 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableWriterExImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableWriterExImpl.java
@@ -1818,10 +1818,10 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx
      */
     private static class SchemaHolder {
         /** Grow step. */
-        private static final int GROW_STEP = 16;
+        private static final int GROW_STEP = 64;
 
         /** Maximum stable size. */
-        private static final int MAX_SIZE = 256;
+        private static final int MAX_SIZE = 1024;
 
         /** Data. */
         private int[] data;

http://git-wip-us.apache.org/repos/asf/ignite/blob/e4b128e3/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
index ffe5d9f..d782aec 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
@@ -258,6 +258,7 @@
     <Compile Include="Impl\Portable\PortableObjectHeader.cs" />
     <Compile Include="Impl\Portable\PortableObjectSchema.cs" />
     <Compile Include="Impl\Portable\PortableObjectSchemaField.cs" />
+    <Compile Include="Impl\Portable\PortableObjectSchemaHolder.cs" />
     <Compile Include="Impl\Portable\PortableReaderExtensions.cs" />
     <Compile Include="Impl\Portable\PortableReaderHandleDictionary.cs" />
     <Compile Include="Impl\Portable\PortableReaderImpl.cs" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/e4b128e3/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/Fnv1Hash.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/Fnv1Hash.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/Fnv1Hash.cs
index 26bbe7c..231220a 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/Fnv1Hash.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/Fnv1Hash.cs
@@ -36,19 +36,22 @@ namespace Apache.Ignite.Core.Impl.Common
         /// <returns>Updated hashcode.</returns>
         public static int Update(int current, int next)
         {
-            current = current ^ (next & 0xFF);
-            current = current * Prime;
+            unchecked
+            {
+                current ^= next & 0xFF;
+                current *= Prime;
 
-            current = current ^ ((next >> 8) & 0xFF);
-            current = current * Prime;
+                current ^= (next >> 8) & 0xFF;
+                current *= Prime;
 
-            current = current ^ ((next >> 16) & 0xFF);
-            current = current * Prime;
+                current ^= (next >> 16) & 0xFF;
+                current *= Prime;
 
-            current = current ^ ((next >> 24) & 0xFF);
-            current = current * Prime;
+                current ^= (next >> 24) & 0xFF;
+                current *= Prime;
 
-            return current;
+                return current;
+            }
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/e4b128e3/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 5e54705..1848f1a 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableBuilderImpl.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableBuilderImpl.cs
@@ -23,7 +23,6 @@ namespace Apache.Ignite.Core.Impl.Portable
     using System.Diagnostics;
     using System.IO;
     using Apache.Ignite.Core.Common;
-    using Apache.Ignite.Core.Impl.Common;
     using Apache.Ignite.Core.Impl.Portable.IO;
     using Apache.Ignite.Core.Impl.Portable.Metadata;
     using Apache.Ignite.Core.Portable;
@@ -626,99 +625,99 @@ namespace Apache.Ignite.Core.Impl.Portable
                         // New object, write in full form.
                         var inSchema = inHeader.ReadSchema(inStream, inStartPos);
 
-                        var outSchemaLen = vals.Count + (inSchema == null ? 0 : inSchema.Length);
-                        var outSchema = outSchemaLen > 0 
-                            ? new ResizeableArray<PortableObjectSchemaField>(outSchemaLen)
-                            : null;
+                        var outSchema = PortableObjectSchemaHolder.Current;
+                        var schemaIdx = outSchema.PushSchema();
 
-                        // Skip header as it is not known at this point.
-                        outStream.Seek(PortableObjectHeader.Size, SeekOrigin.Current);
-
-                        if (inSchema != null)
+                        try
                         {
-                            foreach (var inField in inSchema)
+                            // Skip header as it is not known at this point.
+                            outStream.Seek(PortableObjectHeader.Size, SeekOrigin.Current);
+
+                            if (inSchema != null)
                             {
-                                PortableBuilderField fieldVal;
+                                foreach (var inField in inSchema)
+                                {
+                                    PortableBuilderField fieldVal;
 
-                                var fieldFound = vals.TryGetValue(inField.Id, out fieldVal);
+                                    var fieldFound = vals.TryGetValue(inField.Id, out fieldVal);
 
-                                if (fieldFound && fieldVal == PortableBuilderField.RmvMarker)
-                                    continue;
+                                    if (fieldFound && fieldVal == PortableBuilderField.RmvMarker)
+                                        continue;
 
-                                // ReSharper disable once PossibleNullReferenceException (can't be null)
-                                outSchema.Add(new PortableObjectSchemaField(inField.Id, outStream.Position - outStartPos));
+                                    outSchema.PushField(inField.Id, outStream.Position - outStartPos);
 
-                                if (!fieldFound)
-                                    fieldFound = _parent._cache != null &&
-                                                 _parent._cache.TryGetValue(inField.Offset + inStartPos, out fieldVal);
+                                    if (!fieldFound)
+                                        fieldFound = _parent._cache != null &&
+                                                     _parent._cache.TryGetValue(inField.Offset + inStartPos,
+                                                         out fieldVal);
 
-                                if (fieldFound)
-                                {
-                                    WriteField(ctx, fieldVal);
+                                    if (fieldFound)
+                                    {
+                                        WriteField(ctx, fieldVal);
 
-                                    vals.Remove(inField.Id);
-                                }
-                                else
-                                {
-                                    // Field is not tracked, re-write as is.
-                                    inStream.Seek(inField.Offset + inStartPos, SeekOrigin.Begin);
+                                        vals.Remove(inField.Id);
+                                    }
+                                    else
+                                    {
+                                        // Field is not tracked, re-write as is.
+                                        inStream.Seek(inField.Offset + inStartPos, SeekOrigin.Begin);
 
-                                    Mutate0(ctx, inStream, outStream, false, 0, EmptyVals);
+                                        Mutate0(ctx, inStream, outStream, false, 0, EmptyVals);
+                                    }
                                 }
                             }
-                        }
-
-                        // Write remaining new fields.
-                        foreach (var valEntry in vals)
-                        {
-                            if (valEntry.Value == PortableBuilderField.RmvMarker) 
-                                continue;
-
-                            // ReSharper disable once PossibleNullReferenceException (can't be null)
-                            outSchema.Add(new PortableObjectSchemaField(valEntry.Key, outStream.Position - outStartPos));
 
-                            WriteField(ctx, valEntry.Value);
-                        }
+                            // Write remaining new fields.
+                            foreach (var valEntry in vals)
+                            {
+                                if (valEntry.Value == PortableBuilderField.RmvMarker)
+                                    continue;
 
-                        if (outSchema != null && outSchema.Count == 0)
-                            outSchema = null;
+                                outSchema.PushField(valEntry.Key, outStream.Position - outStartPos);
 
-                        // Write raw data.
-                        int outRawOff = outStream.Position - outStartPos;
+                                WriteField(ctx, valEntry.Value);
+                            }
 
-                        int inRawOff = inHeader.GetRawOffset(inStream, inStartPos);
-                        int inRawLen = inHeader.SchemaOffset - inRawOff;
+                            // Write raw data.
+                            int outRawOff = outStream.Position - outStartPos;
 
-                        if (inRawLen > 0)
-                            outStream.Write(inStream.InternalArray, inStartPos + inRawOff, inRawLen);
+                            int inRawOff = inHeader.GetRawOffset(inStream, inStartPos);
+                            int inRawLen = inHeader.SchemaOffset - inRawOff;
 
-                        // Write schema
-                        int outSchemaOff = outRawOff;
-                        short flags = 0;
+                            if (inRawLen > 0)
+                                outStream.Write(inStream.InternalArray, inStartPos + inRawOff, inRawLen);
 
-                        if (outSchema != null)
-                        {
-                            outSchemaOff = outStream.Position - outStartPos;
+                            // Write schema
+                            int outSchemaOff = outRawOff;
+                            var schemaPos = outStream.Position;
+                            int outSchemaId;
+                            short flags;
 
-                            flags = PortableObjectHeader.WriteSchema(outSchema.Array, outStream, outSchema.Count,
-                                outStream.Position - outStartPos);
+                            var hasSchema = outSchema.WriteSchema(outStream, schemaIdx, out outSchemaId, out flags);
 
-                            if (inRawLen > 0)
-                                outStream.WriteInt(outRawOff);
-                        }
+                            if (hasSchema)
+                            {
+                                outSchemaOff = schemaPos - outStartPos;
 
-                        var outSchemaId = PortableUtils.GetSchemaId(outSchema);
+                                if (inRawLen > 0)
+                                    outStream.WriteInt(outRawOff);
+                            }
 
-                        var outLen = outStream.Position - outStartPos;
+                            var outLen = outStream.Position - outStartPos;
 
-                        var outHash = changeHash ? hash : inHeader.HashCode;
+                            var outHash = changeHash ? hash : inHeader.HashCode;
 
-                        var outHeader = new PortableObjectHeader(inHeader.IsUserType, inHeader.TypeId, outHash, 
-                            outLen, outSchemaId, outSchemaOff, outSchema == null, flags);
+                            var outHeader = new PortableObjectHeader(inHeader.IsUserType, inHeader.TypeId, outHash,
+                                outLen, outSchemaId, outSchemaOff, !hasSchema, flags);
 
-                        PortableObjectHeader.Write(outHeader, outStream, outStartPos);
+                            PortableObjectHeader.Write(outHeader, outStream, outStartPos);
 
-                        outStream.Seek(outStartPos + outLen, SeekOrigin.Begin);  // seek to the end of the object
+                            outStream.Seek(outStartPos + outLen, SeekOrigin.Begin);  // seek to the end of the object
+                        }
+                        finally
+                        {
+                            outSchema.PopSchema(schemaIdx);
+                        }
                     }
                 }
                 else

http://git-wip-us.apache.org/repos/asf/ignite/blob/e4b128e3/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 50adc02..8be8b7f 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableObjectHeader.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableObjectHeader.cs
@@ -294,25 +294,29 @@ namespace Apache.Ignite.Core.Impl.Portable
         /// </summary>
         /// <param name="fields">Fields.</param>
         /// <param name="stream">Stream.</param>
+        /// <param name="offset">Offset in the array.</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.
+        /// 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)
+        public static unsafe short WriteSchema(PortableObjectSchemaField[] fields, IPortableStream stream, int offset,
+            int count)
         {
             Debug.Assert(fields != null);
             Debug.Assert(stream != null);
             Debug.Assert(count > 0);
+            Debug.Assert(offset >= 0);
+            Debug.Assert(offset < fields.Length);
 
             unchecked
             {
-                if (maxOffset <= byte.MaxValue)
+                // Last field is the farthest in the stream
+                var maxFieldOffset = fields[offset + count - 1].Offset;
+
+                if (maxFieldOffset <= byte.MaxValue)
                 {
-                    for (int i = 0; i < count; i++)
+                    for (int i = offset; i < count + offset; i++)
                     {
                         var field = fields[i];
 
@@ -323,9 +327,9 @@ namespace Apache.Ignite.Core.Impl.Portable
                     return FlagByteOffsets;
                 }
 
-                if (maxOffset <= ushort.MaxValue)
+                if (maxFieldOffset <= ushort.MaxValue)
                 {
-                    for (int i = 0; i < count; i++)
+                    for (int i = offset; i < count + offset; i++)
                     {
                         var field = fields[i];
 
@@ -339,14 +343,14 @@ namespace Apache.Ignite.Core.Impl.Portable
 
                 if (BitConverter.IsLittleEndian)
                 {
-                    fixed (PortableObjectSchemaField* ptr = &fields[0])
+                    fixed (PortableObjectSchemaField* ptr = &fields[offset])
                     {
                         stream.Write((byte*)ptr, count / PortableObjectSchemaField.Size);
                     }
                 }
                 else
                 {
-                    for (int i = 0; i < count; i++)
+                    for (int i = offset; i < count + offset; i++)
                     {
                         var field = fields[i];
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/e4b128e3/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableObjectSchemaHolder.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableObjectSchemaHolder.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableObjectSchemaHolder.cs
new file mode 100644
index 0000000..3e99b6e
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableObjectSchemaHolder.cs
@@ -0,0 +1,108 @@
+/*
+ * 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.Threading;
+    using Apache.Ignite.Core.Impl.Common;
+    using Apache.Ignite.Core.Impl.Portable.IO;
+
+    /// <summary>
+    /// Shared schema holder.
+    /// </summary>
+    internal class PortableObjectSchemaHolder
+    {
+        /** Current schema. */
+        private static readonly ThreadLocal<PortableObjectSchemaHolder> CurrentHolder =
+            new ThreadLocal<PortableObjectSchemaHolder>(() => new PortableObjectSchemaHolder());
+
+        /** Fields. */
+        private PortableObjectSchemaField[] _fields = new PortableObjectSchemaField[32];
+
+        /** Current field index. */
+        private int _idx;
+
+        /// <summary>
+        /// Gets the schema holder for the current thread.
+        /// </summary>
+        public static PortableObjectSchemaHolder Current
+        {
+            get { return CurrentHolder.Value; }
+        }
+
+        /// <summary>
+        /// Adds a field to the holder.
+        /// </summary>
+        /// <param name="id">The identifier.</param>
+        /// <param name="offset">The offset.</param>
+        public void PushField(int id, int offset)
+        {
+            if (_idx == _fields.Length)
+                Array.Resize(ref _fields, _fields.Length * 2);
+
+            _fields[_idx] = new PortableObjectSchemaField(id, offset);
+
+            _idx++;
+        }
+
+        /// <summary>
+        /// Gets the start of a new schema
+        /// </summary>
+        public int PushSchema()
+        {
+            return _idx;
+        }
+
+        /// <summary>
+        /// Resets schema position to specified index.
+        /// </summary>
+        public void PopSchema(int idx)
+        {
+            _idx = idx;
+        }
+
+        /// <summary>
+        /// Writes collected schema to the stream and pops it.
+        /// </summary>
+        /// <param name="stream">The stream.</param>
+        /// <param name="schemaOffset">The schema offset.</param>
+        /// <param name="schemaId">The schema identifier.</param>
+        /// <param name="flags">Flags according to offset sizes: <see cref="PortableObjectHeader.FlagByteOffsets" />,
+        /// <see cref="PortableObjectHeader.FlagShortOffsets" />, or 0.</param>
+        /// <returns>
+        /// True if current schema was non empty; false otherwise.
+        /// </returns>
+        public bool WriteSchema(IPortableStream stream, int schemaOffset, out int schemaId, out short flags)
+        {
+            schemaId = Fnv1Hash.Basis;
+            flags = 0;
+
+            var count = _idx - schemaOffset;
+
+            if (count == 0) 
+                return false;
+
+            flags = PortableObjectHeader.WriteSchema(_fields, stream, schemaOffset, count);
+
+            for (var i = schemaOffset; i < _idx; i++)
+                schemaId = Fnv1Hash.Update(schemaId, _fields[i].Id);
+
+            return true;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e4b128e3/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 097cf6d..1255ae3 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableUtils.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Portable/PortableUtils.cs
@@ -1766,28 +1766,6 @@ 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/e4b128e3/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 2dfdfda..ea3e368 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,6 @@ 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;
@@ -70,8 +69,8 @@ namespace Apache.Ignite.Core.Impl.Portable
         /** Current type structure tracker, */
         private PortableStructureTracker _curStruct;
 
-        /** Current schema. */
-        private ResizeableArray<PortableObjectSchemaField> _curSchema;
+        /** Schema holder. */
+        private readonly PortableObjectSchemaHolder _schema = PortableObjectSchemaHolder.Current;
 
         /// <summary>
         /// Gets the marshaller.
@@ -1073,7 +1072,6 @@ namespace Apache.Ignite.Core.Impl.Portable
                 var oldPos = _curPos;
                 
                 var oldStruct = _curStruct;
-                var oldSchema = _curSchema;
 
                 // Push new frame.
                 _curTypeId = desc.TypeId;
@@ -1083,32 +1081,40 @@ namespace Apache.Ignite.Core.Impl.Portable
                 _curPos = pos;
 
                 _curStruct = new PortableStructureTracker(desc, desc.WriterTypeStructure);
-                _curSchema = null;
+                var schemaIdx = _schema.PushSchema();
 
-                // Write object fields.
-                desc.Serializer.WritePortable(obj, this);
+                try
+                {
+                    // Write object fields.
+                    desc.Serializer.WritePortable(obj, this);
 
-                // Write schema
-                var hasSchema = _curSchema != null;
-                var schemaOffset = hasSchema ? _stream.Position - pos : PortableObjectHeader.Size;
-                short flags = 0;
+                    // Write schema
+                    var schemaOffset = _stream.Position - pos;
 
-                if (hasSchema)
-                    flags = PortableObjectHeader.WriteSchema(_curSchema.Array, _stream, _curSchema.Count,
-                        _curSchema.Array[_curSchema.Count - 1].Offset);
+                    int schemaId;
+                    short flags;
+                    var hasSchema = _schema.WriteSchema(_stream, schemaIdx, out schemaId, out flags);
 
-                // Calculate and write header.
-                if (hasSchema && _curRawPos > 0)
-                    _stream.WriteInt(_curRawPos - pos); // raw offset is in the last 4 bytes
+                    if (!hasSchema)
+                        schemaOffset = PortableObjectHeader.Size;
 
-                var len = _stream.Position - pos;
+                    // Calculate and write header.
+                    if (hasSchema && _curRawPos > 0)
+                        _stream.WriteInt(_curRawPos - pos); // raw offset is in the last 4 bytes
 
-                var header = new PortableObjectHeader(desc.UserType, desc.TypeId, obj.GetHashCode(), len,
-                    PU.GetSchemaId(_curSchema), schemaOffset, !hasSchema, flags);
+                    var len = _stream.Position - pos;
 
-                PortableObjectHeader.Write(header, _stream, pos);
+                    var header = new PortableObjectHeader(desc.UserType, desc.TypeId, obj.GetHashCode(), len,
+                        schemaId, schemaOffset, !hasSchema, flags);
 
-                Stream.Seek(pos + len, SeekOrigin.Begin);  // Seek to the end
+                    PortableObjectHeader.Write(header, _stream, pos);
+
+                    Stream.Seek(pos + len, SeekOrigin.Begin); // Seek to the end
+                }
+                finally
+                {
+                    _schema.PopSchema(schemaIdx);
+                }
 
                 // Apply structure updates if any.
                 _curStruct.UpdateWriterStructure(this);
@@ -1121,7 +1127,6 @@ namespace Apache.Ignite.Core.Impl.Portable
                 _curPos = oldPos;
 
                 _curStruct = oldStruct;
-                _curSchema = oldSchema;
             }
             else
             {
@@ -1373,9 +1378,7 @@ namespace Apache.Ignite.Core.Impl.Portable
 
             var fieldId = _curStruct.GetFieldId(fieldName, fieldTypeId);
 
-            _curSchema = _curSchema ?? new ResizeableArray<PortableObjectSchemaField>(4);
-
-            _curSchema.Add(new PortableObjectSchemaField(fieldId, _stream.Position - _curPos));
+            _schema.PushField(fieldId, _stream.Position - _curPos);
         }
 
         /// <summary>