You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vo...@apache.org on 2015/11/12 09:01:37 UTC

[1/3] ignite git commit: IGNITE-1816: Debugging.

Repository: ignite
Updated Branches:
  refs/heads/ignite-1816 ed3472d4e -> d5dbf632a


IGNITE-1816: Debugging.


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

Branch: refs/heads/ignite-1816
Commit: 25c264fb5c1781a3ec76b5c7c92716243b66e2b2
Parents: ed3472d
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Wed Nov 11 16:57:32 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Wed Nov 11 16:57:32 2015 +0300

----------------------------------------------------------------------
 .../internal/portable/BinaryReaderExImpl.java   | 28 ++++++--
 .../internal/portable/BinaryWriterExImpl.java   | 60 ++++++++++------
 .../internal/portable/PortableSchema.java       | 76 ++++++++++++++++----
 .../builder/BinaryObjectBuilderImpl.java        |  1 -
 .../ignite/internal/util/IgniteUtils.java       | 25 +++++++
 .../marshaller/portable/PortableMarshaller.java |  3 +-
 6 files changed, 149 insertions(+), 44 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/25c264fb/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryReaderExImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryReaderExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryReaderExImpl.java
index ff33d08..d487ae1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryReaderExImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryReaderExImpl.java
@@ -2620,11 +2620,29 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Obje
                 schema0 = ctx.schemaRegistry(typeId).schema(schemaId);
 
                 if (schema0 == null) {
-                    if (fieldIdLen == 0)
-                        throw new BinaryObjectException("Cannot find schema for object without field IDs [" +
-                            "typeId=" + typeId + ", schemaId=" + schemaId + ']');
+                    if (fieldIdLen == 0) {
+                        BinaryTypeImpl type = (BinaryTypeImpl)ctx.metaData(typeId);
 
-                    schema0 = createSchema();
+                        if (type == null || type.metadata() == null)
+                            throw new BinaryObjectException("Cannot find metadata for object with compact footer: " +
+                                typeId);
+
+                        for (PortableSchema typeSchema : type.metadata().schemas()) {
+                            if (schemaId == typeSchema.schemaId()) {
+                                schema0 = typeSchema;
+
+                                break;
+                            }
+                        }
+
+                        if (schema0 == null)
+                            throw new BinaryObjectException("Cannot find schema for object with compact fotter [" +
+                                "typeId=" + typeId + ", schemaId=" + schemaId + ']');
+                    }
+                    else
+                        schema0 = createSchema();
+
+                    assert schema0 != null;
 
                     ctx.schemaRegistry(typeId).addSchema(schemaId, schema0);
                 }
@@ -2632,8 +2650,6 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Obje
                 schema = schema0;
             }
 
-            assert schema != null;
-
             int order = schema.order(id);
 
             if (order != PortableSchema.ORDER_NOT_FOUND) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/25c264fb/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryWriterExImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryWriterExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryWriterExImpl.java
index 23d9311..d23bc69 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryWriterExImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryWriterExImpl.java
@@ -347,7 +347,7 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
             out.writeInt(start + SCHEMA_OR_RAW_OFF_POS, out.position() - start);
 
             // Write the schema.
-            int offsetByteCnt = schema.write(this, fieldCnt, !ctx.isCompactFooter());
+            int offsetByteCnt = schema.write(this, fieldCnt, ctx.isCompactFooter());
 
             // Write raw offset if needed.
             if (rawOffPos != 0)
@@ -1853,10 +1853,10 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
          *
          * @param writer Writer.
          * @param fieldCnt Count.
-         * @param writeFieldId Whether to write field IDs.
+         * @param compactFooter Whether footer should be written in compact form.
          * @return Amount of bytes dedicated to each field offset. Could be 1, 2 or 4.
          */
-        public int write(BinaryWriterExImpl writer, int fieldCnt, boolean writeFieldId) {
+        public int write(BinaryWriterExImpl writer, int fieldCnt, boolean compactFooter) {
             int startIdx = idx - fieldCnt * 2;
 
             assert startIdx >= 0;
@@ -1865,35 +1865,51 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
 
             int res;
 
-            if (lastOffset < MAX_OFFSET_1) {
-                for (int idx0 = startIdx; idx0 < idx; ) {
-                    if (writeFieldId)
-                        writer.writeInt(data[idx0++]);
+            if (compactFooter) {
+                if (lastOffset < MAX_OFFSET_1) {
+                    for (int curIdx = startIdx + 1; curIdx < idx; curIdx += 2)
+                        writer.writeByte((byte)data[curIdx]);
 
-                    writer.writeByte((byte) data[idx0++]);
+                    res = PortableUtils.OFFSET_1;
                 }
+                else if (lastOffset < MAX_OFFSET_2) {
+                    for (int curIdx = startIdx + 1; curIdx < idx; curIdx += 2)
+                        writer.writeShort((short)data[curIdx]);
 
-                res = PortableUtils.OFFSET_1;
-            }
-            else if (lastOffset < MAX_OFFSET_2) {
-                for (int idx0 = startIdx; idx0 < idx; ) {
-                    if (writeFieldId)
-                        writer.writeInt(data[idx0++]);
-
-                    writer.writeShort((short)data[idx0++]);
+                    res = PortableUtils.OFFSET_2;
                 }
+                else {
+                    for (int curIdx = startIdx + 1; curIdx < idx; curIdx += 2)
+                        writer.writeInt(data[curIdx]);
 
-                res = PortableUtils.OFFSET_2;
+                    res = PortableUtils.OFFSET_4;
+                }
             }
             else {
-                for (int idx0 = startIdx; idx0 < idx; ) {
-                    if (writeFieldId)
-                        writer.writeInt(data[idx0++]);
+                if (lastOffset < MAX_OFFSET_1) {
+                    for (int curIdx = startIdx; curIdx < idx;) {
+                        writer.writeInt(data[curIdx++]);
+                        writer.writeByte((byte) data[curIdx++]);
+                    }
 
-                    writer.writeInt(data[idx0++]);
+                    res = PortableUtils.OFFSET_1;
                 }
+                else if (lastOffset < MAX_OFFSET_2) {
+                    for (int curIdx = startIdx; curIdx < idx;) {
+                        writer.writeInt(data[curIdx++]);
+                        writer.writeShort((short)data[curIdx++]);
+                    }
 
-                res = PortableUtils.OFFSET_4;
+                    res = PortableUtils.OFFSET_2;
+                }
+                else {
+                    for (int curIdx = startIdx; curIdx < idx;) {
+                        writer.writeInt(data[curIdx++]);
+                        writer.writeInt(data[curIdx++]);
+                    }
+
+                    res = PortableUtils.OFFSET_4;
+                }
             }
 
             return res;

http://git-wip-us.apache.org/repos/asf/ignite/blob/25c264fb/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableSchema.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableSchema.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableSchema.java
index f6838c0..3cb191e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableSchema.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableSchema.java
@@ -17,7 +17,12 @@
 
 package org.apache.ignite.internal.portable;
 
-import java.io.Serializable;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.Iterator;
@@ -29,7 +34,7 @@ import java.util.List;
  * for quick comparisons performed within already fetched L1 cache line.
  * - When there are more fields, we store them inside a hash map.
  */
-public class PortableSchema implements Serializable {
+public class PortableSchema implements Externalizable {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -37,37 +42,82 @@ public class PortableSchema implements Serializable {
     public static final int ORDER_NOT_FOUND = -1;
 
     /** Inline flag. */
-    private final boolean inline;
+    private boolean inline;
 
     /** Map with offsets. */
-    private final HashMap<Integer, Integer> map;
+    private HashMap<Integer, Integer> map;
 
     /** ID 1. */
-    private final int id0;
+    private int id0;
 
     /** ID 2. */
-    private final int id1;
+    private int id1;
 
     /** ID 3. */
-    private final int id2;
+    private int id2;
 
     /** ID 4. */
-    private final int id3;
+    private int id3;
 
     /** ID 1. */
-    private final int id4;
+    private int id4;
 
     /** ID 2. */
-    private final int id5;
+    private int id5;
 
     /** ID 3. */
-    private final int id6;
+    private int id6;
 
     /** ID 4. */
-    private final int id7;
+    private int id7;
 
     /** Schema ID. */
-    private final int schemaId;
+    private int schemaId;
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeInt(schemaId);
+
+        if (inline) {
+            out.writeBoolean(true);
+
+            out.writeInt(id0);
+            out.writeInt(id1);
+            out.writeInt(id2);
+            out.writeInt(id3);
+            out.writeInt(id4);
+            out.writeInt(id5);
+            out.writeInt(id6);
+            out.writeInt(id7);
+        }
+        else {
+            out.writeBoolean(false);
+            U.writeMap(out, map);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        schemaId = in.readInt();
+
+        if (in.readBoolean()) {
+            inline = true;
+
+            id0 = in.readInt();
+            id1 = in.readInt();
+            id2 = in.readInt();
+            id3 = in.readInt();
+            id4 = in.readInt();
+            id5 = in.readInt();
+            id6 = in.readInt();
+            id7 = in.readInt();
+        }
+        else {
+            inline = false;
+
+            map = U.readHashMap(in);
+        }
+    }
 
     /**
      * Constructor.

http://git-wip-us.apache.org/repos/asf/ignite/blob/25c264fb/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/BinaryObjectBuilderImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/BinaryObjectBuilderImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/BinaryObjectBuilderImpl.java
index 402363b..268df1a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/BinaryObjectBuilderImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/BinaryObjectBuilderImpl.java
@@ -177,7 +177,6 @@ public class BinaryObjectBuilderImpl implements BinaryObjectBuilder {
     /** {@inheritDoc} */
     @Override public BinaryObject build() {
         try (BinaryWriterExImpl writer = new BinaryWriterExImpl(ctx, typeId, false)) {
-
             PortableBuilderSerializer serializationCtx = new PortableBuilderSerializer();
 
             serializationCtx.registerObjectWriting(this, 0);

http://git-wip-us.apache.org/repos/asf/ignite/blob/25c264fb/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
index 259d8c9..7337378 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
@@ -4983,6 +4983,31 @@ public abstract class IgniteUtils {
     }
 
     /**
+     * Read hash map.
+     *
+     * @param in Input.
+     * @return Read map.
+     * @throws IOException If de-serialization failed.
+     * @throws ClassNotFoundException If deserialized class could not be found.
+     */
+    @SuppressWarnings({"unchecked"})
+    @Nullable public static <K, V> HashMap<K, V> readHashMap(ObjectInput in)
+        throws IOException, ClassNotFoundException {
+        int size = in.readInt();
+
+        // Check null flag.
+        if (size == -1)
+            return null;
+
+        HashMap<K, V> map = U.newHashMap(size);
+
+        for (int i = 0; i < size; i++)
+            map.put((K)in.readObject(), (V)in.readObject());
+
+        return map;
+    }
+
+    /**
      *
      * @param in Input.
      * @return Read map.

http://git-wip-us.apache.org/repos/asf/ignite/blob/25c264fb/modules/core/src/main/java/org/apache/ignite/marshaller/portable/PortableMarshaller.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/portable/PortableMarshaller.java b/modules/core/src/main/java/org/apache/ignite/marshaller/portable/PortableMarshaller.java
index 4e8217d..1704c8a 100644
--- a/modules/core/src/main/java/org/apache/ignite/marshaller/portable/PortableMarshaller.java
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/portable/PortableMarshaller.java
@@ -78,8 +78,7 @@ public class PortableMarshaller extends AbstractMarshaller {
     public static final boolean DFLT_KEEP_DESERIALIZED = true;
 
     /** Default value of "compact footer" flag. */
-    // TODO: Set to true.
-    public static final boolean DFLT_COMPACT_FOOTER = false;
+    public static final boolean DFLT_COMPACT_FOOTER = true;
 
     // TODO ignite-1282 Move to IgniteConfiguration.
     /** Class names. */


[3/3] ignite git commit: IGNITE-1816: WIP on tests.

Posted by vo...@apache.org.
IGNITE-1816: WIP on tests.


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

Branch: refs/heads/ignite-1816
Commit: d5dbf632a5505af545fce45b9fd771a61760fc0a
Parents: 66d9ffc
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Thu Nov 12 11:02:20 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Nov 12 11:02:20 2015 +0300

----------------------------------------------------------------------
 .../marshaller/portable/PortableMarshaller.java       |  2 +-
 .../portable/GridPortableMarshallerSelfTest.java      | 14 +++++++++-----
 .../internal/portable/TestCachingMetadataHandler.java |  8 ++++----
 3 files changed, 14 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d5dbf632/modules/core/src/main/java/org/apache/ignite/marshaller/portable/PortableMarshaller.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/portable/PortableMarshaller.java b/modules/core/src/main/java/org/apache/ignite/marshaller/portable/PortableMarshaller.java
index b84ba68..1704c8a 100644
--- a/modules/core/src/main/java/org/apache/ignite/marshaller/portable/PortableMarshaller.java
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/portable/PortableMarshaller.java
@@ -78,7 +78,7 @@ public class PortableMarshaller extends AbstractMarshaller {
     public static final boolean DFLT_KEEP_DESERIALIZED = true;
 
     /** Default value of "compact footer" flag. */
-    public static final boolean DFLT_COMPACT_FOOTER = false;
+    public static final boolean DFLT_COMPACT_FOOTER = true;
 
     // TODO ignite-1282 Move to IgniteConfiguration.
     /** Class names. */

http://git-wip-us.apache.org/repos/asf/ignite/blob/d5dbf632/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java
index 6c80767..69d0d42 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java
@@ -546,7 +546,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
             new BinaryTypeConfiguration(TestBinary.class.getName())
         ));
 
-        TestBinary obj = BinaryObject();
+        TestBinary obj = binaryObject();
 
         BinaryObject po = marshal(obj, marsh);
 
@@ -899,6 +899,8 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
             new BinaryTypeConfiguration(DynamicObject.class.getName())
         ));
 
+        initializePortableContext(marsh);
+
         BinaryObject po1 = marshal(new DynamicObject(0, 10, 20, 30), marsh);
 
         assertEquals(new Integer(10), po1.field("val1"));
@@ -1092,7 +1094,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
             customMappingType
         ));
 
-        TestBinary obj = BinaryObject();
+        TestBinary obj = binaryObject();
 
         BinaryObjectImpl po = marshal(obj, marsh1);
 
@@ -1307,10 +1309,14 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
             new BinaryTypeConfiguration(SimpleObject.class.getName())
         ));
 
+        initializePortableContext(marsh);
+
         SimpleObject obj = simpleObject();
 
         final BinaryObject po = marshal(obj, marsh);
 
+        assertEquals(obj, po.deserialize());
+
         BinaryObject copy = copy(po, null);
 
         assertEquals(obj, copy.deserialize());
@@ -2547,7 +2553,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
     /**
      * @return Portable object.
      */
-    private TestBinary BinaryObject() {
+    private TestBinary binaryObject() {
         SimpleObject innerSimple = new SimpleObject();
 
         innerSimple.b = 1;
@@ -3389,8 +3395,6 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
 
             if (idx > 1)
                 writer.writeInt("val3", val3);
-
-            idx++;
         }
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/d5dbf632/modules/core/src/test/java/org/apache/ignite/internal/portable/TestCachingMetadataHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/TestCachingMetadataHandler.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/TestCachingMetadataHandler.java
index bb80781..f863317 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/TestCachingMetadataHandler.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/TestCachingMetadataHandler.java
@@ -20,17 +20,17 @@ package org.apache.ignite.internal.portable;
 import org.apache.ignite.binary.BinaryObjectException;
 import org.apache.ignite.binary.BinaryType;
 
-import java.util.concurrent.ConcurrentHashMap;
+import java.util.HashMap;
 
 /**
  * Test metadata handler.
  */
 public class TestCachingMetadataHandler implements BinaryMetadataHandler {
     /** Cached metadatas. */
-    private final ConcurrentHashMap<Integer, BinaryType> metas = new ConcurrentHashMap<>();
+    private final HashMap<Integer, BinaryType> metas = new HashMap<>();
 
     /** {@inheritDoc} */
-    @Override public void addMeta(int typeId, BinaryType type) throws BinaryObjectException {
+    @Override public synchronized void addMeta(int typeId, BinaryType type) throws BinaryObjectException {
         synchronized (this) {
             BinaryType oldType = metas.put(typeId, type);
 
@@ -48,7 +48,7 @@ public class TestCachingMetadataHandler implements BinaryMetadataHandler {
     }
 
     /** {@inheritDoc} */
-    @Override public BinaryType metadata(int typeId) throws BinaryObjectException {
+    @Override public synchronized BinaryType metadata(int typeId) throws BinaryObjectException {
         return metas.get(typeId);
     }
 }


[2/3] ignite git commit: IGNITE-1816: WIP on tests.

Posted by vo...@apache.org.
IGNITE-1816: WIP on tests.


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

Branch: refs/heads/ignite-1816
Commit: 66d9ffc58d302ecf11c8279b75ae4b80268ddd7f
Parents: 25c264f
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Wed Nov 11 17:28:48 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Wed Nov 11 17:28:48 2015 +0300

----------------------------------------------------------------------
 .../internal/portable/BinaryTypeImpl.java       |   8 +-
 .../ignite/internal/portable/PortableUtils.java |  68 +++++++
 .../CacheObjectBinaryProcessorImpl.java         |  83 +--------
 .../marshaller/portable/PortableMarshaller.java |   2 +-
 .../GridPortableMarshallerSelfTest.java         | 184 ++++++++++---------
 .../portable/TestCachingMetadataHandler.java    |  19 +-
 6 files changed, 198 insertions(+), 166 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/66d9ffc5/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryTypeImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryTypeImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryTypeImpl.java
index 40b6252..2630a40 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryTypeImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryTypeImpl.java
@@ -62,10 +62,16 @@ public class BinaryTypeImpl implements BinaryType {
         return ctx.createField(meta.typeId(), fieldName);
     }
 
-    public String affinityKeyFieldName() {
+    /** {@inheritDoc} */
+    @Override public String affinityKeyFieldName() {
         return meta.affinityKeyFieldName();
     }
 
+    /** {@inheritDoc} */
+    public PortableContext context() {
+        return ctx;
+    }
+
     /**
      * @return Metadata.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/66d9ffc5/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableUtils.java
index cb445c5..b49cc5b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableUtils.java
@@ -807,4 +807,72 @@ public class PortableUtils {
 
         return res;
     }
+
+    /**
+     * Merge old and new metas.
+     *
+     * @param oldMeta Old meta.
+     * @param newMeta New meta.
+     * @return New meta if old meta was null, old meta if no changes detected, merged meta otherwise.
+     * @throws BinaryObjectException If merge failed due to metadata conflict.
+     */
+    public static BinaryMetadata mergeMetadata(@Nullable BinaryMetadata oldMeta, BinaryMetadata newMeta) {
+        assert newMeta != null;
+
+        if (oldMeta == null)
+            return newMeta;
+        else {
+            assert oldMeta.typeId() == newMeta.typeId();
+
+            // Check type name.
+            if (!F.eq(oldMeta.typeName(), newMeta.typeName())) {
+                throw new BinaryObjectException(
+                    "Two portable types have duplicate type ID [" + "typeId=" + oldMeta.typeId() +
+                        ", typeName1=" + oldMeta.typeName() + ", typeName2=" + newMeta.typeName() + ']'
+                );
+            }
+
+            // Check affinity field names.
+            if (!F.eq(oldMeta.affinityKeyFieldName(), newMeta.affinityKeyFieldName())) {
+                throw new BinaryObjectException(
+                    "Binary type has different affinity key fields [" + "typeName=" + newMeta.typeName() +
+                        ", affKeyFieldName1=" + oldMeta.affinityKeyFieldName() +
+                        ", affKeyFieldName2=" + newMeta.affinityKeyFieldName() + ']'
+                );
+            }
+
+            // Check and merge fields.
+            boolean changed = false;
+
+            Map<String, Integer> mergedFields = new HashMap<>(oldMeta.fieldsMap());
+            Map<String, Integer> newFields = newMeta.fieldsMap();
+
+            for (Map.Entry<String, Integer> newField : newFields.entrySet()) {
+                Integer oldFieldType = mergedFields.put(newField.getKey(), newField.getValue());
+
+                if (oldFieldType == null)
+                    changed = true;
+                else if (!F.eq(oldFieldType, newField.getValue())) {
+                    throw new BinaryObjectException(
+                        "Binary type has different field types [" + "typeName=" + oldMeta.typeName() +
+                            ", fieldName=" + newField.getKey() +
+                            ", fieldTypeName1=" + PortableUtils.fieldTypeName(oldFieldType) +
+                            ", fieldTypeName2=" + PortableUtils.fieldTypeName(newField.getValue()) + ']'
+                    );
+                }
+            }
+
+            // Check and merge schemas.
+            Collection<PortableSchema> mergedSchemas = new HashSet<>(oldMeta.schemas());
+
+            for (PortableSchema newSchema : newMeta.schemas()) {
+                if (mergedSchemas.add(newSchema))
+                    changed = true;
+            }
+
+            // Return either old meta if no changes detected, or new merged meta.
+            return changed ? new BinaryMetadata(oldMeta.typeId(), oldMeta.typeName(), mergedFields,
+                oldMeta.affinityKeyFieldName(), mergedSchemas) : oldMeta;
+        }
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/66d9ffc5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java
index 01ec71c..eb51d13 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java
@@ -31,13 +31,12 @@ import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
 import org.apache.ignite.internal.portable.BinaryMetadata;
+import org.apache.ignite.internal.portable.BinaryMetadataHandler;
 import org.apache.ignite.internal.portable.BinaryObjectImpl;
 import org.apache.ignite.internal.portable.BinaryObjectOffheapImpl;
 import org.apache.ignite.internal.portable.BinaryTypeImpl;
 import org.apache.ignite.internal.portable.GridPortableMarshaller;
 import org.apache.ignite.internal.portable.PortableContext;
-import org.apache.ignite.internal.portable.BinaryMetadataHandler;
-import org.apache.ignite.internal.portable.PortableSchema;
 import org.apache.ignite.internal.portable.PortableUtils;
 import org.apache.ignite.internal.portable.builder.BinaryObjectBuilderImpl;
 import org.apache.ignite.internal.portable.streams.PortableInputStream;
@@ -89,8 +88,6 @@ import java.io.ObjectInput;
 import java.io.ObjectOutput;
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.HashMap;
-import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
 import java.util.UUID;
@@ -170,11 +167,11 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
 
                     if (metaDataCache == null) {
                         BinaryMetadata oldMeta = metaBuf.get(typeId);
-                        BinaryMetadata mergedMeta = mergeMetadata(oldMeta, newMeta0);
+                        BinaryMetadata mergedMeta = PortableUtils.mergeMetadata(oldMeta, newMeta0);
 
                         if (oldMeta != mergedMeta) {
                             synchronized (this) {
-                                mergedMeta = mergeMetadata(oldMeta, newMeta0);
+                                mergedMeta = PortableUtils.mergeMetadata(oldMeta, newMeta0);
 
                                 if (oldMeta != mergedMeta)
                                     metaBuf.put(typeId, mergedMeta);
@@ -301,7 +298,7 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
                 BinaryMetadata oldMeta0 = oldMeta != null ? oldMeta.metadata() : null;
 
                 try {
-                    res = mergeMetadata(oldMeta0, newMeta);
+                    res = PortableUtils.mergeMetadata(oldMeta0, newMeta);
                 }
                 catch (BinaryObjectException e) {
                     res = oldMeta0;
@@ -459,7 +456,7 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
 
         try {
             BinaryMetadata oldMeta = metaDataCache.localPeek(key);
-            BinaryMetadata mergedMeta = mergeMetadata(oldMeta, newMeta0);
+            BinaryMetadata mergedMeta = PortableUtils.mergeMetadata(oldMeta, newMeta0);
 
             BinaryObjectException err = metaDataCache.invoke(key, new MetadataProcessor(mergedMeta));
 
@@ -721,74 +718,6 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
     }
 
     /**
-     * Merge old and new metas.
-     *
-     * @param oldMeta Old meta.
-     * @param newMeta New meta.
-     * @return New meta if old meta was null, old meta if no changes detected, merged meta otherwise.
-     * @throws BinaryObjectException If merge failed due to metadata conflict.
-     */
-    private static BinaryMetadata mergeMetadata(@Nullable BinaryMetadata oldMeta, BinaryMetadata newMeta) {
-        assert newMeta != null;
-
-        if (oldMeta == null)
-            return newMeta;
-        else {
-            assert oldMeta.typeId() == newMeta.typeId();
-
-            // Check type name.
-            if (!F.eq(oldMeta.typeName(), newMeta.typeName())) {
-                throw new BinaryObjectException(
-                    "Two portable types have duplicate type ID [" + "typeId=" + oldMeta.typeId() +
-                        ", typeName1=" + oldMeta.typeName() + ", typeName2=" + newMeta.typeName() + ']'
-                );
-            }
-
-            // Check affinity field names.
-            if (!F.eq(oldMeta.affinityKeyFieldName(), newMeta.affinityKeyFieldName())) {
-                throw new BinaryObjectException(
-                    "Binary type has different affinity key fields [" + "typeName=" + newMeta.typeName() +
-                        ", affKeyFieldName1=" + oldMeta.affinityKeyFieldName() +
-                        ", affKeyFieldName2=" + newMeta.affinityKeyFieldName() + ']'
-                );
-            }
-
-            // Check and merge fields.
-            boolean changed = false;
-
-            Map<String, Integer> mergedFields = new HashMap<>(oldMeta.fieldsMap());
-            Map<String, Integer> newFields = newMeta.fieldsMap();
-
-            for (Map.Entry<String, Integer> newField : newFields.entrySet()) {
-                Integer oldFieldType = mergedFields.put(newField.getKey(), newField.getValue());
-
-                if (oldFieldType == null)
-                    changed = true;
-                else if (!F.eq(oldFieldType, newField.getValue())) {
-                    throw new BinaryObjectException(
-                        "Binary type has different field types [" + "typeName=" + oldMeta.typeName() +
-                            ", fieldName=" + newField.getKey() +
-                            ", fieldTypeName1=" + PortableUtils.fieldTypeName(oldFieldType) +
-                            ", fieldTypeName2=" + PortableUtils.fieldTypeName(newField.getValue()) + ']'
-                    );
-                }
-            }
-
-            // Check and merge schemas.
-            Collection<PortableSchema> mergedSchemas = new HashSet<>(oldMeta.schemas());
-
-            for (PortableSchema newSchema : newMeta.schemas()) {
-                if (mergedSchemas.add(newSchema))
-                    changed = true;
-            }
-
-            // Return either old meta if no changes detected, or new merged meta.
-            return changed ? new BinaryMetadata(oldMeta.typeId(), oldMeta.typeName(), mergedFields,
-                oldMeta.affinityKeyFieldName(), mergedSchemas) : oldMeta;
-        }
-    }
-
-    /**
      * Processor responsible for metadata update.
      */
     private static class MetadataProcessor
@@ -821,7 +750,7 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
             try {
                 BinaryMetadata oldMeta = entry.getValue();
 
-                BinaryMetadata mergedMeta = mergeMetadata(oldMeta, newMeta);
+                BinaryMetadata mergedMeta = PortableUtils.mergeMetadata(oldMeta, newMeta);
 
                 if (mergedMeta != oldMeta)
                     entry.setValue(mergedMeta);

http://git-wip-us.apache.org/repos/asf/ignite/blob/66d9ffc5/modules/core/src/main/java/org/apache/ignite/marshaller/portable/PortableMarshaller.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/portable/PortableMarshaller.java b/modules/core/src/main/java/org/apache/ignite/marshaller/portable/PortableMarshaller.java
index 1704c8a..b84ba68 100644
--- a/modules/core/src/main/java/org/apache/ignite/marshaller/portable/PortableMarshaller.java
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/portable/PortableMarshaller.java
@@ -78,7 +78,7 @@ public class PortableMarshaller extends AbstractMarshaller {
     public static final boolean DFLT_KEEP_DESERIALIZED = true;
 
     /** Default value of "compact footer" flag. */
-    public static final boolean DFLT_COMPACT_FOOTER = true;
+    public static final boolean DFLT_COMPACT_FOOTER = false;
 
     // TODO ignite-1282 Move to IgniteConfiguration.
     /** Class names. */

http://git-wip-us.apache.org/repos/asf/ignite/blob/66d9ffc5/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java
index d0a5709..6c80767 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java
@@ -48,7 +48,6 @@ import org.apache.ignite.binary.BinaryRawReader;
 import org.apache.ignite.binary.BinaryRawWriter;
 import org.apache.ignite.binary.BinaryReader;
 import org.apache.ignite.binary.BinarySerializer;
-import org.apache.ignite.binary.BinaryType;
 import org.apache.ignite.binary.BinaryTypeConfiguration;
 import org.apache.ignite.binary.BinaryIdMapper;
 import org.apache.ignite.binary.BinaryWriter;
@@ -384,7 +383,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testBinaryObject() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
+        PortableMarshaller marsh = createMarshaller();
 
         marsh.setTypeConfigurations(Arrays.asList(new BinaryTypeConfiguration(SimpleObject.class.getName())));
 
@@ -413,7 +412,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testEnum() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
+        PortableMarshaller marsh = createMarshaller();
 
         marsh.setClassNames(Arrays.asList(TestEnum.class.getName()));
 
@@ -426,7 +425,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
     public void testDateAndTimestampInSingleObject() throws Exception {
         BinaryTypeConfiguration cfg1 = new BinaryTypeConfiguration(DateClass1.class.getName());
 
-        PortableMarshaller marsh = new PortableMarshaller();
+        PortableMarshaller marsh = createMarshaller();
 
         marsh.setTypeConfigurations(Arrays.asList(cfg1));
 
@@ -453,7 +452,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testSimpleObject() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
+        PortableMarshaller marsh = createMarshaller();
 
         marsh.setTypeConfigurations(Arrays.asList(
             new BinaryTypeConfiguration(SimpleObject.class.getName())
@@ -540,7 +539,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testPortable() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
+        PortableMarshaller marsh = createMarshaller();
 
         marsh.setTypeConfigurations(Arrays.asList(
             new BinaryTypeConfiguration(SimpleObject.class.getName()),
@@ -704,7 +703,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testClassWithoutPublicConstructor() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
+        PortableMarshaller marsh = createMarshaller();
 
         marsh.setTypeConfigurations(Arrays.asList(
                 new BinaryTypeConfiguration(NoPublicConstructor.class.getName()),
@@ -712,8 +711,6 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
                 new BinaryTypeConfiguration(ProtectedConstructor.class.getName()))
         );
 
-        initPortableContext(marsh);
-
         NoPublicConstructor npc = new NoPublicConstructor();
         BinaryObject npc2 = marshal(npc, marsh);
 
@@ -734,7 +731,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testCustomSerializer() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
+        PortableMarshaller marsh = createMarshaller();
 
         BinaryTypeConfiguration type =
             new BinaryTypeConfiguration(CustomSerializedObject1.class.getName());
@@ -754,7 +751,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testCustomSerializerWithGlobal() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
+        PortableMarshaller marsh = createMarshaller();
 
         marsh.setSerializer(new CustomSerializer1());
 
@@ -784,7 +781,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testCustomIdMapper() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
+        PortableMarshaller marsh = createMarshaller();
 
         BinaryTypeConfiguration type =
             new BinaryTypeConfiguration(CustomMappedObject1.class.getName());
@@ -826,7 +823,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testCustomIdMapperWithGlobal() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
+        PortableMarshaller marsh = createMarshaller();
 
         marsh.setIdMapper(new BinaryIdMapper() {
             @Override public int typeId(String clsName) {
@@ -896,7 +893,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testDynamicObject() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
+        PortableMarshaller marsh = createMarshaller();
 
         marsh.setTypeConfigurations(Arrays.asList(
             new BinaryTypeConfiguration(DynamicObject.class.getName())
@@ -943,7 +940,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testCycleLink() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
+        PortableMarshaller marsh = createMarshaller();
 
         marsh.setTypeConfigurations(Arrays.asList(
             new BinaryTypeConfiguration(CycleLinkObject.class.getName())
@@ -964,7 +961,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testDetached() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
+        PortableMarshaller marsh = createMarshaller();
 
         marsh.setTypeConfigurations(Arrays.asList(
             new BinaryTypeConfiguration(DetachedTestObject.class.getName()),
@@ -1020,7 +1017,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testCollectionFields() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
+        PortableMarshaller marsh = createMarshaller();
 
         marsh.setTypeConfigurations(Arrays.asList(
             new BinaryTypeConfiguration(CollectionFieldsObject.class.getName()),
@@ -1062,7 +1059,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testDefaultMapping() throws Exception {
-        PortableMarshaller marsh1 = new PortableMarshaller();
+        PortableMarshaller marsh1 = createMarshaller();
 
         BinaryTypeConfiguration customMappingType =
             new BinaryTypeConfiguration(TestBinary.class.getName());
@@ -1099,14 +1096,14 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
 
         BinaryObjectImpl po = marshal(obj, marsh1);
 
-        PortableMarshaller marsh2 = new PortableMarshaller();
+        PortableMarshaller marsh2 = createMarshaller();
 
         marsh2.setTypeConfigurations(Arrays.asList(
             new BinaryTypeConfiguration(SimpleObject.class.getName()),
             new BinaryTypeConfiguration(TestBinary.class.getName())
         ));
 
-        PortableContext ctx = initPortableContext(marsh2);
+        PortableContext ctx = initializePortableContext(marsh2);
 
         po.context(ctx);
 
@@ -1117,7 +1114,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testTypeNames() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
+        PortableMarshaller marsh = createMarshaller();
 
         BinaryTypeConfiguration customType1 = new BinaryTypeConfiguration(Value.class.getName());
 
@@ -1177,7 +1174,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
             customType4
         ));
 
-        PortableContext ctx = initPortableContext(marsh);
+        PortableContext ctx = initializePortableContext(marsh);
 
         assertEquals("notconfiguredclass".hashCode(), ctx.typeId("NotConfiguredClass"));
         assertEquals("key".hashCode(), ctx.typeId("Key"));
@@ -1193,7 +1190,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testFieldIdMapping() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
+        PortableMarshaller marsh = createMarshaller();
 
         BinaryTypeConfiguration customType1 = new BinaryTypeConfiguration(Value.class.getName());
 
@@ -1242,7 +1239,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
             customType1,
             customType2));
 
-        PortableContext ctx = initPortableContext(marsh);
+        PortableContext ctx = initializePortableContext(marsh);
 
         assertEquals("val".hashCode(), ctx.fieldId("key".hashCode(), "val"));
         assertEquals("val".hashCode(), ctx.fieldId("nonexistentclass2".hashCode(), "val"));
@@ -1259,7 +1256,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testDuplicateTypeId() throws Exception {
-        final PortableMarshaller marsh = new PortableMarshaller();
+        final PortableMarshaller marsh = createMarshaller();
 
         BinaryTypeConfiguration customType1 = new BinaryTypeConfiguration("org.gridgain.Class1");
 
@@ -1288,7 +1285,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         marsh.setTypeConfigurations(Arrays.asList(customType1, customType2));
 
         try {
-            initPortableContext(marsh);
+            initializePortableContext(marsh);
         }
         catch (IgniteCheckedException e) {
             assertEquals("Duplicate type ID [clsName=org.gridgain.Class1, id=100]",
@@ -1304,7 +1301,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testPortableCopy() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
+        PortableMarshaller marsh = createMarshaller();
 
         marsh.setTypeConfigurations(Arrays.asList(
             new BinaryTypeConfiguration(SimpleObject.class.getName())
@@ -1427,7 +1424,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testPortableCopyString() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
+        PortableMarshaller marsh = createMarshaller();
 
         marsh.setTypeConfigurations(Arrays.asList(
             new BinaryTypeConfiguration(SimpleObject.class.getName())
@@ -1450,7 +1447,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testPortableCopyUuid() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
+        PortableMarshaller marsh = createMarshaller();
 
         marsh.setTypeConfigurations(Arrays.asList(
             new BinaryTypeConfiguration(SimpleObject.class.getName())
@@ -1475,7 +1472,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testPortableCopyByteArray() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
+        PortableMarshaller marsh = createMarshaller();
 
         marsh.setTypeConfigurations(Arrays.asList(
             new BinaryTypeConfiguration(SimpleObject.class.getName())
@@ -1514,7 +1511,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testPortableCopyShortArray() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
+        PortableMarshaller marsh = createMarshaller();
 
         marsh.setTypeConfigurations(Arrays.asList(
             new BinaryTypeConfiguration(SimpleObject.class.getName())
@@ -1537,7 +1534,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testPortableCopyIntArray() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
+        PortableMarshaller marsh = createMarshaller();
 
         marsh.setTypeConfigurations(Arrays.asList(
             new BinaryTypeConfiguration(SimpleObject.class.getName())
@@ -1560,7 +1557,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testPortableCopyLongArray() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
+        PortableMarshaller marsh = createMarshaller();
 
         marsh.setTypeConfigurations(Arrays.asList(
             new BinaryTypeConfiguration(SimpleObject.class.getName())
@@ -1583,7 +1580,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testPortableCopyFloatArray() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
+        PortableMarshaller marsh = createMarshaller();
 
         marsh.setTypeConfigurations(Arrays.asList(
             new BinaryTypeConfiguration(SimpleObject.class.getName())
@@ -1606,7 +1603,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testPortableCopyDoubleArray() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
+        PortableMarshaller marsh = createMarshaller();
 
         marsh.setTypeConfigurations(Arrays.asList(
             new BinaryTypeConfiguration(SimpleObject.class.getName())
@@ -1629,7 +1626,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testPortableCopyCharArray() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
+        PortableMarshaller marsh = createMarshaller();
 
         marsh.setTypeConfigurations(Arrays.asList(
             new BinaryTypeConfiguration(SimpleObject.class.getName())
@@ -1652,7 +1649,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testPortableCopyStringArray() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
+        PortableMarshaller marsh = createMarshaller();
 
         marsh.setTypeConfigurations(Arrays.asList(
             new BinaryTypeConfiguration(SimpleObject.class.getName())
@@ -1675,7 +1672,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testPortableCopyObject() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
+        PortableMarshaller marsh = createMarshaller();
 
         marsh.setTypeConfigurations(Arrays.asList(
             new BinaryTypeConfiguration(SimpleObject.class.getName())
@@ -1704,7 +1701,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testPortableCopyNonPrimitives() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
+        PortableMarshaller marsh = createMarshaller();
 
         marsh.setTypeConfigurations(Arrays.asList(
             new BinaryTypeConfiguration(SimpleObject.class.getName())
@@ -1743,7 +1740,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testPortableCopyMixed() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
+        PortableMarshaller marsh = createMarshaller();
 
         marsh.setTypeConfigurations(Arrays.asList(new BinaryTypeConfiguration(SimpleObject.class.getName())));
 
@@ -1789,7 +1786,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testKeepDeserialized() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
+        PortableMarshaller marsh = createMarshaller();
 
         marsh.setClassNames(Arrays.asList(SimpleObject.class.getName()));
         marsh.setKeepDeserialized(true);
@@ -1798,7 +1795,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
 
         assert po.deserialize() == po.deserialize();
 
-        marsh = new PortableMarshaller();
+        marsh = createMarshaller();
 
         marsh.setClassNames(Arrays.asList(SimpleObject.class.getName()));
         marsh.setKeepDeserialized(false);
@@ -1807,7 +1804,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
 
         assert po.deserialize() != po.deserialize();
 
-        marsh = new PortableMarshaller();
+        marsh = createMarshaller();
 
         marsh.setKeepDeserialized(true);
         marsh.setTypeConfigurations(Arrays.asList(
@@ -1817,7 +1814,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
 
         assert po.deserialize() == po.deserialize();
 
-        marsh = new PortableMarshaller();
+        marsh = createMarshaller();
 
         marsh.setKeepDeserialized(false);
         marsh.setTypeConfigurations(Arrays.asList(
@@ -1827,7 +1824,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
 
         assert po.deserialize() != po.deserialize();
 
-        marsh = new PortableMarshaller();
+        marsh = createMarshaller();
 
         marsh.setKeepDeserialized(true);
 
@@ -1841,7 +1838,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
 
         assert po.deserialize() != po.deserialize();
 
-        marsh = new PortableMarshaller();
+        marsh = createMarshaller();
 
         marsh.setKeepDeserialized(false);
 
@@ -1860,11 +1857,11 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testOffheapPortable() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
+        PortableMarshaller marsh = createMarshaller();
 
         marsh.setTypeConfigurations(Arrays.asList(new BinaryTypeConfiguration(SimpleObject.class.getName())));
 
-        PortableContext ctx = initPortableContext(marsh);
+        PortableContext ctx = initializePortableContext(marsh);
 
         SimpleObject simpleObj = simpleObject();
 
@@ -1956,7 +1953,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
      *
      */
     public void testReadResolve() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
+        PortableMarshaller marsh = createMarshaller();
 
         marsh.setClassNames(
             Arrays.asList(MySingleton.class.getName(), SingletonMarker.class.getName()));
@@ -1974,7 +1971,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
      *
      */
     public void testReadResolveOnPortableAware() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
+        PortableMarshaller marsh = createMarshaller();
 
         marsh.setClassNames(Collections.singletonList(MyTestClass.class.getName()));
 
@@ -1989,7 +1986,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
      * @throws Exception If ecxeption thrown.
      */
     public void testDeclareReadResolveInParent() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
+        PortableMarshaller marsh = createMarshaller();
 
         marsh.setClassNames(Arrays.asList(ChildPortable.class.getName()));
 
@@ -2004,7 +2001,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
      *
      */
     public void testDecimalFields() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
+        PortableMarshaller marsh = createMarshaller();
 
         Collection<String> clsNames = new ArrayList<>();
 
@@ -2049,8 +2046,8 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
     /**
      * @throws IgniteCheckedException If failed.
      */
-    public void testFinalField() throws IgniteCheckedException {
-        PortableMarshaller marsh = new PortableMarshaller();
+    public void testFinalField() throws Exception {
+        PortableMarshaller marsh = createMarshaller();
 
         SimpleObjectWithFinal obj = new SimpleObjectWithFinal();
 
@@ -2062,11 +2059,13 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
     /**
      * @throws IgniteCheckedException If failed.
      */
-    public void testThreadLocalArrayReleased() throws IgniteCheckedException {
+    public void testThreadLocalArrayReleased() throws Exception {
         // Checking the writer directly.
         assertEquals(false, THREAD_LOCAL_ALLOC.isThreadLocalArrayAcquired());
 
-        try (BinaryWriterExImpl writer = new BinaryWriterExImpl(initPortableContext(new PortableMarshaller()))) {
+        PortableMarshaller marsh0 = createMarshaller();
+
+        try (BinaryWriterExImpl writer = new BinaryWriterExImpl(portableContext(marsh0))) {
             assertEquals(true, THREAD_LOCAL_ALLOC.isThreadLocalArrayAcquired());
 
             writer.writeString("Thread local test");
@@ -2079,16 +2078,16 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         // Checking the portable marshaller.
         assertEquals(false, THREAD_LOCAL_ALLOC.isThreadLocalArrayAcquired());
 
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        initPortableContext(marsh);
+        PortableMarshaller marsh = createMarshaller();
 
         marsh.marshal(new SimpleObject());
 
         assertEquals(false, THREAD_LOCAL_ALLOC.isThreadLocalArrayAcquired());
 
         // Checking the builder.
-        BinaryObjectBuilder builder = new BinaryObjectBuilderImpl(initPortableContext(new PortableMarshaller()),
+        PortableMarshaller marsh2 = createMarshaller();
+
+        BinaryObjectBuilder builder = new BinaryObjectBuilderImpl(portableContext(marsh2),
             "org.gridgain.foo.bar.TestClass");
 
         builder.setField("a", "1");
@@ -2102,9 +2101,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testDuplicateName() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        initPortableContext(marsh);
+        PortableMarshaller marsh = createMarshaller();
 
         Test1.Job job1 = new Test1().new Job();
         Test2.Job job2 = new Test2().new Job();
@@ -2113,8 +2110,10 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
 
         try {
             marsh.marshal(job2);
-        } catch (BinaryObjectException e) {
+        }
+        catch (BinaryObjectException e) {
             assertEquals(true, e.getMessage().contains("Failed to register class"));
+
             return;
         }
 
@@ -2125,9 +2124,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testClass() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        initPortableContext(marsh);
+        PortableMarshaller marsh = createMarshaller();
 
         Class cls = GridPortableMarshallerSelfTest.class;
 
@@ -2140,9 +2137,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testClassFieldsMarshalling() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        initPortableContext(marsh);
+        PortableMarshaller marsh = createMarshaller();
 
         ObjectWithClassFields obj = new ObjectWithClassFields();
         obj.cls1 = GridPortableMarshallerSelfTest.class;
@@ -2165,9 +2160,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testMarshallingThroughJdk() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        initPortableContext(marsh);
+        PortableMarshaller marsh = createMarshaller();
 
         InetSocketAddress addr = new InetSocketAddress("192.168.0.2", 4545);
 
@@ -2203,9 +2196,9 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testPredefinedTypeIds() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
+        PortableMarshaller marsh = createMarshaller();
 
-        PortableContext pCtx = initPortableContext(marsh);
+        PortableContext pCtx = initializePortableContext(marsh);
 
         Field field = pCtx.getClass().getDeclaredField("predefinedTypeNames");
 
@@ -2232,7 +2225,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testCyclicReferencesMarshalling() throws Exception {
-        PortableMarshaller marsh = new PortableMarshaller();
+        PortableMarshaller marsh = createMarshaller();
 
         SimpleObject obj = simpleObject();
 
@@ -2366,8 +2359,8 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
      * @param obj Original object.
      * @return Result object.
      */
-    private <T> T marshalUnmarshal(T obj) throws IgniteCheckedException {
-        return marshalUnmarshal(obj, new PortableMarshaller());
+    private <T> T marshalUnmarshal(T obj) throws Exception {
+        return marshalUnmarshal(obj, createMarshaller());
     }
 
     /**
@@ -2376,7 +2369,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
      * @return Result object.
      */
     private <T> T marshalUnmarshal(Object obj, PortableMarshaller marsh) throws IgniteCheckedException {
-        initPortableContext(marsh);
+        initializePortableContext(marsh);
 
         byte[] bytes = marsh.marshal(obj);
 
@@ -2389,7 +2382,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
      * @return Portable object.
      */
     private <T> BinaryObjectImpl marshal(T obj, PortableMarshaller marsh) throws IgniteCheckedException {
-        initPortableContext(marsh);
+        initializePortableContext(marsh);
 
         byte[] bytes = marsh.marshal(obj);
 
@@ -2398,12 +2391,39 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * Create portable marshaller.
+     *
+     * @return Portable marshaller.
+     * @throws Exception If failed.
+     */
+    private PortableMarshaller createMarshaller() throws Exception {
+        PortableMarshaller marsh = new PortableMarshaller();
+
+        initializePortableContext(marsh);
+
+        return marsh;
+    }
+
+    /**
+     * Get portable context of the given marshaller.
+     *
+     * @param marsh Marshaller.
+     * @return Context.
+     * @throws Exception If failed.
+     */
+    private PortableContext portableContext(PortableMarshaller marsh) throws Exception {
+        GridPortableMarshaller marsh0 = IgniteUtils.field(marsh, "impl");
+
+        return marsh0.context();
+    }
+
+    /**
      * @return Portable context.
      */
-    protected PortableContext initPortableContext(PortableMarshaller marsh) throws IgniteCheckedException {
+    private PortableContext initializePortableContext(PortableMarshaller marsh) throws IgniteCheckedException {
         IgniteConfiguration iCfg = new IgniteConfiguration();
 
-        PortableContext ctx = new PortableContext(BinaryNoopMetadataHandler.instance(), iCfg);
+        PortableContext ctx = new PortableContext(new TestCachingMetadataHandler(), iCfg);
 
         marsh.setContext(new MarshallerContextTestImpl(null));
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/66d9ffc5/modules/core/src/test/java/org/apache/ignite/internal/portable/TestCachingMetadataHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/TestCachingMetadataHandler.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/TestCachingMetadataHandler.java
index e49ebf3..bb80781 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/TestCachingMetadataHandler.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/TestCachingMetadataHandler.java
@@ -30,12 +30,21 @@ public class TestCachingMetadataHandler implements BinaryMetadataHandler {
     private final ConcurrentHashMap<Integer, BinaryType> metas = new ConcurrentHashMap<>();
 
     /** {@inheritDoc} */
-    @Override public void addMeta(int typeId, BinaryType meta) throws BinaryObjectException {
-        BinaryType otherType = metas.put(typeId, meta);
+    @Override public void addMeta(int typeId, BinaryType type) throws BinaryObjectException {
+        synchronized (this) {
+            BinaryType oldType = metas.put(typeId, type);
 
-        if (otherType != null)
-            throw new IllegalStateException("Metadata replacement is not allowed in " +
-                TestCachingMetadataHandler.class.getSimpleName() + '.');
+            if (oldType != null) {
+                BinaryMetadata oldMeta = ((BinaryTypeImpl)oldType).metadata();
+                BinaryMetadata newMeta = ((BinaryTypeImpl)type).metadata();
+
+                BinaryMetadata mergedMeta = PortableUtils.mergeMetadata(oldMeta, newMeta);
+
+                BinaryType mergedType = mergedMeta.wrap(((BinaryTypeImpl)oldType).context());
+
+                metas.put(typeId, mergedType);
+            }
+        }
     }
 
     /** {@inheritDoc} */