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/21 23:45:42 UTC

[14/55] [abbrv] ignite git commit: IGNITE-1816: Implemented compact footers.

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableCompactOffsetsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableCompactOffsetsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableCompactOffsetsAbstractSelfTest.java
deleted file mode 100644
index 9225b97..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableCompactOffsetsAbstractSelfTest.java
+++ /dev/null
@@ -1,190 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.portable;
-
-import org.apache.ignite.binary.BinaryField;
-import org.apache.ignite.binary.BinaryTypeConfiguration;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.util.IgniteUtils;
-import org.apache.ignite.marshaller.MarshallerContextTestImpl;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-
-import java.util.Arrays;
-
-/**
- * Contains tests for compact offsets.
- */
-public abstract class PortableCompactOffsetsAbstractSelfTest extends GridCommonAbstractTest {
-    /** 2 pow 8. */
-    private static int POW_8 = 1 << 8;
-
-    /** 2 pow 16. */
-    private static int POW_16 = 1 << 16;
-
-    /** Marshaller. */
-    protected PortableMarshaller marsh;
-
-    /** Portable context. */
-    protected PortableContext ctx;
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTest() throws Exception {
-        super.beforeTest();
-
-        ctx = new PortableContext(new TestCachingMetadataHandler(), new IgniteConfiguration());
-
-        marsh = new PortableMarshaller();
-
-        marsh.setTypeConfigurations(Arrays.asList(new BinaryTypeConfiguration(TestObject.class.getName())));
-        marsh.setContext(new MarshallerContextTestImpl(null));
-
-        IgniteUtils.invoke(PortableMarshaller.class, marsh, "setPortableContext", ctx);
-    }
-
-    /**
-     * Test 1 byte.
-     *
-     * @throws Exception If failed.
-     */
-    public void test1Byte() throws Exception {
-        check(POW_8 >> 2);
-    }
-
-    /**
-     * Test 1 byte with sign altering.
-     *
-     * @throws Exception If failed.
-     */
-    public void test1ByteSign() throws Exception {
-        check(POW_8 >> 1);
-    }
-
-    /**
-     * Test 2 bytes.
-     *
-     * @throws Exception If failed.
-     */
-    public void test2Bytes() throws Exception {
-        check(POW_16 >> 2);
-    }
-
-    /**
-     * Test 2 bytes with sign altering.
-     *
-     * @throws Exception If failed.
-     */
-    public void test2BytesSign() throws Exception {
-        check(POW_16 >> 1);
-    }
-
-    /**
-     * Test 4 bytes.
-     *
-     * @throws Exception If failed.
-     */
-    public void test4Bytes() throws Exception {
-        check(POW_16 << 2);
-    }
-
-    /**
-     * Main check routine.
-     *
-     * @param len Length of the first field.
-     *
-     * @throws Exception If failed.
-     */
-    private void check(int len) throws Exception {
-        TestObject obj = new TestObject(len);
-
-        BinaryObjectEx portObj = toPortable(marsh, obj);
-
-        // 1. Test portable object content.
-        assert portObj.hasField("field1");
-        assert portObj.hasField("field2");
-
-        byte[] field1 = portObj.field("field1");
-        Integer field2 = portObj.field("field2");
-
-        assert field1 != null;
-        assert field2 != null;
-
-        assert Arrays.equals(obj.field1, field1);
-        assert obj.field2 == field2;
-
-        // 2. Test fields API.
-        BinaryField field1Desc = portObj.type().field("field1");
-        BinaryField field2Desc = portObj.type().field("field2");
-
-        assert field1Desc.exists(portObj);
-        assert field2Desc.exists(portObj);
-
-        assert Arrays.equals(obj.field1, (byte[])field1Desc.value(portObj));
-        assert obj.field2 == (Integer)field2Desc.value(portObj);
-
-        // 3. Test deserialize.
-        TestObject objRestored = portObj.deserialize();
-
-        assert objRestored != null;
-
-        assert Arrays.equals(obj.field1, objRestored.field1);
-        assert obj.field2 == objRestored.field2;
-    }
-
-    /**
-     * Convert object to portable object.
-     *
-     * @param marsh Marshaller.
-     * @param obj Object.
-     * @return Portable object.
-     * @throws Exception If failed.
-     */
-    protected abstract BinaryObjectEx toPortable(PortableMarshaller marsh, Object obj) throws Exception;
-
-    /**
-     * Test object.
-     */
-    public static class TestObject {
-        /** First field with variable length. */
-        public byte[] field1;
-
-        /** Second field. */
-        public int field2;
-
-        /**
-         * Default constructor.
-         */
-        public TestObject() {
-            // No-op.
-        }
-
-        /**
-         * Constructor.
-         *
-         * @param len Array length.
-         */
-        public TestObject(int len) {
-            field1 = new byte[len];
-
-            field1[0] = 1;
-            field1[len - 1] = 2;
-
-            field2 = len;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableCompactOffsetsHeapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableCompactOffsetsHeapSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableCompactOffsetsHeapSelfTest.java
deleted file mode 100644
index ebdef38..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableCompactOffsetsHeapSelfTest.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.portable;
-
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
-
-/**
- * Compact offsets tests for heap portable objects.
- */
-public class PortableCompactOffsetsHeapSelfTest extends PortableCompactOffsetsAbstractSelfTest {
-    /** {@inheritDoc} */
-    @Override protected BinaryObjectEx toPortable(PortableMarshaller marsh, Object obj) throws Exception {
-        byte[] bytes = marsh.marshal(obj);
-
-        return new BinaryObjectImpl(ctx, bytes, 0);
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableCompactOffsetsOffheapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableCompactOffsetsOffheapSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableCompactOffsetsOffheapSelfTest.java
deleted file mode 100644
index e3b6bda..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableCompactOffsetsOffheapSelfTest.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.portable;
-
-import org.apache.ignite.internal.util.GridUnsafe;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.eclipse.jetty.util.ConcurrentHashSet;
-import sun.misc.Unsafe;
-
-/**
- * Compact offsets tests for offheap portable objects.
- */
-public class PortableCompactOffsetsOffheapSelfTest extends PortableCompactOffsetsAbstractSelfTest {
-    /** Unsafe instance. */
-    private static final Unsafe UNSAFE = GridUnsafe.unsafe();
-
-    /** Byte array offset for unsafe mechanics. */
-    protected static final long BYTE_ARR_OFF = UNSAFE.arrayBaseOffset(byte[].class);
-
-    /** Allocated unsafe pointer. */
-    private final ConcurrentHashSet<Long> ptrs = new ConcurrentHashSet<>();
-
-    /** {@inheritDoc} */
-    @Override protected void afterTest() throws Exception {
-        super.afterTest();
-
-        // Cleanup allocated objects.
-        for (Long ptr : ptrs)
-            UNSAFE.freeMemory(ptr);
-
-        ptrs.clear();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected BinaryObjectEx toPortable(PortableMarshaller marsh, Object obj) throws Exception {
-        byte[] arr = marsh.marshal(obj);
-
-        long ptr = UNSAFE.allocateMemory(arr.length);
-
-        ptrs.add(ptr);
-
-        UNSAFE.copyMemory(arr, BYTE_ARR_OFF, null, ptr, arr.length);
-
-        return new BinaryObjectOffheapImpl(ctx, ptr, 0, arr.length);
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/core/src/test/java/org/apache/ignite/internal/portable/noncompact/BinaryFieldsHeapNonCompactSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/noncompact/BinaryFieldsHeapNonCompactSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/noncompact/BinaryFieldsHeapNonCompactSelfTest.java
new file mode 100644
index 0000000..9e7619f
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/noncompact/BinaryFieldsHeapNonCompactSelfTest.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.portable.noncompact;
+
+import org.apache.ignite.internal.portable.BinaryFieldsAbstractSelfTest;
+import org.apache.ignite.internal.portable.BinaryFieldsHeapSelfTest;
+import org.apache.ignite.internal.portable.BinaryObjectEx;
+import org.apache.ignite.internal.portable.BinaryObjectImpl;
+import org.apache.ignite.marshaller.portable.PortableMarshaller;
+
+/**
+ * Field tests for heap-based portables with non-compact footer.
+ */
+public class BinaryFieldsHeapNonCompactSelfTest extends BinaryFieldsHeapSelfTest {
+    /** {@inheritDoc} */
+    @Override protected boolean compactFooter() {
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/core/src/test/java/org/apache/ignite/internal/portable/noncompact/BinaryFieldsOffheapNonCompactSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/noncompact/BinaryFieldsOffheapNonCompactSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/noncompact/BinaryFieldsOffheapNonCompactSelfTest.java
new file mode 100644
index 0000000..0bca601
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/noncompact/BinaryFieldsOffheapNonCompactSelfTest.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.portable.noncompact;
+
+import org.apache.ignite.internal.portable.BinaryFieldsOffheapSelfTest;
+
+/**
+ * Field tests for offheap-based portables with non-compact footer.
+ */
+public class BinaryFieldsOffheapNonCompactSelfTest extends BinaryFieldsOffheapSelfTest {
+    /** {@inheritDoc} */
+    @Override protected boolean compactFooter() {
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/core/src/test/java/org/apache/ignite/internal/portable/noncompact/BinaryFooterOffsetsHeapNonCompactSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/noncompact/BinaryFooterOffsetsHeapNonCompactSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/noncompact/BinaryFooterOffsetsHeapNonCompactSelfTest.java
new file mode 100644
index 0000000..8fba738
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/noncompact/BinaryFooterOffsetsHeapNonCompactSelfTest.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.portable.noncompact;
+
+import org.apache.ignite.internal.portable.BinaryFooterOffsetsHeapSelfTest;
+
+/**
+ * Compact offsets tests for heap portable objects with non-compact footer.
+ */
+public class BinaryFooterOffsetsHeapNonCompactSelfTest extends BinaryFooterOffsetsHeapSelfTest {
+    /** {@inheritDoc} */
+    @Override protected boolean compactFooter() {
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/core/src/test/java/org/apache/ignite/internal/portable/noncompact/BinaryFooterOffsetsOffheapNonCompactSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/noncompact/BinaryFooterOffsetsOffheapNonCompactSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/noncompact/BinaryFooterOffsetsOffheapNonCompactSelfTest.java
new file mode 100644
index 0000000..b52bd83
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/noncompact/BinaryFooterOffsetsOffheapNonCompactSelfTest.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.portable.noncompact;
+
+import org.apache.ignite.internal.portable.BinaryFooterOffsetsOffheapSelfTest;
+
+/**
+ * Compact offsets tests for offheap portable objects with non-compact footer.
+ */
+public class BinaryFooterOffsetsOffheapNonCompactSelfTest extends BinaryFooterOffsetsOffheapSelfTest {
+    /** {@inheritDoc} */
+    @Override protected boolean compactFooter() {
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/core/src/test/java/org/apache/ignite/internal/portable/noncompact/BinaryMarshallerNonCompactSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/noncompact/BinaryMarshallerNonCompactSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/noncompact/BinaryMarshallerNonCompactSelfTest.java
new file mode 100644
index 0000000..0484dea
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/noncompact/BinaryMarshallerNonCompactSelfTest.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.portable.noncompact;
+
+import org.apache.ignite.internal.portable.BinaryMarshallerSelfTest;
+
+/**
+ * Basic marshaller test with non-compact footer.
+ */
+public class BinaryMarshallerNonCompactSelfTest extends BinaryMarshallerSelfTest {
+    /** {@inheritDoc} */
+    @Override protected boolean compactFooter() {
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/core/src/test/java/org/apache/ignite/internal/portable/noncompact/BinaryObjectBuilderAdditionalNonCompactSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/noncompact/BinaryObjectBuilderAdditionalNonCompactSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/noncompact/BinaryObjectBuilderAdditionalNonCompactSelfTest.java
new file mode 100644
index 0000000..8811029
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/noncompact/BinaryObjectBuilderAdditionalNonCompactSelfTest.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.portable.noncompact;
+
+import org.apache.ignite.internal.portable.BinaryObjectBuilderAdditionalSelfTest;
+
+/**
+ *
+ */
+public class BinaryObjectBuilderAdditionalNonCompactSelfTest extends BinaryObjectBuilderAdditionalSelfTest {
+    /** {@inheritDoc} */
+    @Override protected boolean compactFooter() {
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/core/src/test/java/org/apache/ignite/internal/portable/noncompact/BinaryObjectBuilderNonCompactSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/noncompact/BinaryObjectBuilderNonCompactSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/noncompact/BinaryObjectBuilderNonCompactSelfTest.java
new file mode 100644
index 0000000..0b0916d
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/noncompact/BinaryObjectBuilderNonCompactSelfTest.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.portable.noncompact;
+
+import org.apache.ignite.internal.portable.BinaryObjectBuilderSelfTest;
+
+/**
+ * Portable builder test for objects with non-compact footer.
+ */
+public class BinaryObjectBuilderNonCompactSelfTest extends BinaryObjectBuilderSelfTest {
+    /** {@inheritDoc} */
+    @Override protected boolean compactFooter() {
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePortableObjectsTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePortableObjectsTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePortableObjectsTestSuite.java
index 4fe8633..1128d67 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePortableObjectsTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePortableObjectsTestSuite.java
@@ -19,16 +19,23 @@ package org.apache.ignite.testsuites;
 
 import junit.framework.TestSuite;
 import org.apache.ignite.internal.portable.GridPortableAffinityKeySelfTest;
-import org.apache.ignite.internal.portable.GridBinaryObjectBuilderAdditionalSelfTest;
-import org.apache.ignite.internal.portable.GridBinaryObjectBuilderSelfTest;
+import org.apache.ignite.internal.portable.BinaryObjectBuilderAdditionalSelfTest;
+import org.apache.ignite.internal.portable.BinaryObjectBuilderSelfTest;
 import org.apache.ignite.internal.portable.GridPortableMarshallerCtxDisabledSelfTest;
-import org.apache.ignite.internal.portable.GridPortableMarshallerSelfTest;
+import org.apache.ignite.internal.portable.BinaryMarshallerSelfTest;
 import org.apache.ignite.internal.portable.GridPortableMetaDataSelfTest;
 import org.apache.ignite.internal.portable.GridPortableWildcardsSelfTest;
-import org.apache.ignite.internal.portable.PortableCompactOffsetsHeapSelfTest;
-import org.apache.ignite.internal.portable.PortableCompactOffsetsOffheapSelfTest;
+import org.apache.ignite.internal.portable.BinaryFooterOffsetsHeapSelfTest;
+import org.apache.ignite.internal.portable.BinaryFooterOffsetsOffheapSelfTest;
 import org.apache.ignite.internal.portable.BinaryFieldsHeapSelfTest;
 import org.apache.ignite.internal.portable.BinaryFieldsOffheapSelfTest;
+import org.apache.ignite.internal.portable.noncompact.BinaryFieldsHeapNonCompactSelfTest;
+import org.apache.ignite.internal.portable.noncompact.BinaryFieldsOffheapNonCompactSelfTest;
+import org.apache.ignite.internal.portable.noncompact.BinaryFooterOffsetsHeapNonCompactSelfTest;
+import org.apache.ignite.internal.portable.noncompact.BinaryFooterOffsetsOffheapNonCompactSelfTest;
+import org.apache.ignite.internal.portable.noncompact.BinaryMarshallerNonCompactSelfTest;
+import org.apache.ignite.internal.portable.noncompact.BinaryObjectBuilderAdditionalNonCompactSelfTest;
+import org.apache.ignite.internal.portable.noncompact.BinaryObjectBuilderNonCompactSelfTest;
 import org.apache.ignite.internal.processors.cache.portable.GridCacheClientNodeBinaryObjectMetadataMultinodeTest;
 import org.apache.ignite.internal.processors.cache.portable.GridCacheClientNodeBinaryObjectMetadataTest;
 import org.apache.ignite.internal.processors.cache.portable.GridCachePortableStoreObjectsSelfTest;
@@ -57,18 +64,27 @@ public class IgnitePortableObjectsTestSuite extends TestSuite {
     public static TestSuite suite() throws Exception {
         TestSuite suite = new TestSuite("GridGain Portable Objects Test Suite");
 
-        suite.addTestSuite(GridPortableMarshallerSelfTest.class);
+        suite.addTestSuite(BinaryMarshallerSelfTest.class);
         suite.addTestSuite(GridPortableMarshallerCtxDisabledSelfTest.class);
-        suite.addTestSuite(GridBinaryObjectBuilderSelfTest.class);
-        suite.addTestSuite(GridBinaryObjectBuilderAdditionalSelfTest.class);
+        suite.addTestSuite(BinaryObjectBuilderSelfTest.class);
+        suite.addTestSuite(BinaryObjectBuilderAdditionalSelfTest.class);
         suite.addTestSuite(BinaryFieldsHeapSelfTest.class);
         suite.addTestSuite(BinaryFieldsOffheapSelfTest.class);
-        suite.addTestSuite(PortableCompactOffsetsHeapSelfTest.class);
-        suite.addTestSuite(PortableCompactOffsetsOffheapSelfTest.class);
+        suite.addTestSuite(BinaryFooterOffsetsHeapSelfTest.class);
+        suite.addTestSuite(BinaryFooterOffsetsOffheapSelfTest.class);
         suite.addTestSuite(GridPortableMetaDataSelfTest.class);
         suite.addTestSuite(GridPortableAffinityKeySelfTest.class);
         suite.addTestSuite(GridPortableWildcardsSelfTest.class);
 
+        // Tests for objects with non-compact footers.
+        suite.addTestSuite(BinaryMarshallerNonCompactSelfTest.class);
+        suite.addTestSuite(BinaryObjectBuilderNonCompactSelfTest.class);
+        suite.addTestSuite(BinaryObjectBuilderAdditionalNonCompactSelfTest.class);
+        suite.addTestSuite(BinaryFieldsHeapNonCompactSelfTest.class);
+        suite.addTestSuite(BinaryFieldsOffheapNonCompactSelfTest.class);
+        suite.addTestSuite(BinaryFooterOffsetsHeapNonCompactSelfTest.class);
+        suite.addTestSuite(BinaryFooterOffsetsOffheapNonCompactSelfTest.class);
+
         suite.addTestSuite(GridCacheBinaryObjectsLocalSelfTest.class);
         suite.addTestSuite(GridCacheBinaryObjectsAtomicLocalSelfTest.class);
         suite.addTestSuite(GridCacheBinaryObjectsReplicatedSelfTest.class);

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/platforms/cpp/core-test/src/binary_reader_writer_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/src/binary_reader_writer_test.cpp b/modules/platforms/cpp/core-test/src/binary_reader_writer_test.cpp
index d273b11..b2551ec 100644
--- a/modules/platforms/cpp/core-test/src/binary_reader_writer_test.cpp
+++ b/modules/platforms/cpp/core-test/src/binary_reader_writer_test.cpp
@@ -70,7 +70,7 @@ void CheckPrimitive(T val)
     int32_t footerBegin = in.ReadInt32(IGNITE_OFFSET_SCHEMA_OR_RAW_OFF);
     int32_t footerEnd = footerBegin + 5;
 
-    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_1_BYTE);
+    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_ONE_BYTE);
     BinaryReader reader(&readerImpl);
 
     try
@@ -150,7 +150,7 @@ void CheckPrimitiveArray(T dflt, T val1, T val2)
         int32_t footerBegin = in.ReadInt32(IGNITE_OFFSET_SCHEMA_OR_RAW_OFF);
         int32_t footerEnd = footerBegin + 5;
 
-        BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_1_BYTE);
+        BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_ONE_BYTE);
         BinaryReader reader(&readerImpl);
 
         in.Position(IGNITE_DFLT_HDR_LEN);
@@ -183,7 +183,7 @@ void CheckPrimitiveArray(T dflt, T val1, T val2)
         int32_t footerBegin = in.ReadInt32(IGNITE_OFFSET_SCHEMA_OR_RAW_OFF);
         int32_t footerEnd = footerBegin + 5;
 
-        BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_1_BYTE);
+        BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_ONE_BYTE);
         BinaryReader reader(&readerImpl);
 
         in.Position(IGNITE_DFLT_HDR_LEN);
@@ -220,7 +220,7 @@ void CheckPrimitiveArray(T dflt, T val1, T val2)
         int32_t footerBegin = in.ReadInt32(IGNITE_OFFSET_SCHEMA_OR_RAW_OFF);
         int32_t footerEnd = footerBegin + 5;
 
-        BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_1_BYTE);
+        BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_ONE_BYTE);
         BinaryReader reader(&readerImpl);
 
         in.Position(IGNITE_DFLT_HDR_LEN);
@@ -260,7 +260,7 @@ void CheckPrimitiveArray(T dflt, T val1, T val2)
         int32_t footerBegin = in.ReadInt32(IGNITE_OFFSET_SCHEMA_OR_RAW_OFF);
         int32_t footerEnd = footerBegin + 5;
 
-        BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_1_BYTE);
+        BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_ONE_BYTE);
         BinaryReader reader(&readerImpl);
 
         in.Position(IGNITE_DFLT_HDR_LEN);
@@ -512,7 +512,7 @@ void CheckCollectionEmpty(CollectionType* colType)
     int32_t footerBegin = in.ReadInt32(IGNITE_OFFSET_SCHEMA_OR_RAW_OFF);
     int32_t footerEnd = footerBegin + 5 * 2;
 
-    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_1_BYTE);
+    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_ONE_BYTE);
     BinaryReader reader(&readerImpl);
 
     in.Position(IGNITE_DFLT_HDR_LEN);
@@ -602,7 +602,7 @@ void CheckCollection(CollectionType* colType)
     int32_t footerBegin = in.ReadInt32(IGNITE_OFFSET_SCHEMA_OR_RAW_OFF);
     int32_t footerEnd = footerBegin + 5 * 2;
 
-    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_1_BYTE);
+    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_ONE_BYTE);
     BinaryReader reader(&readerImpl);
 
     in.Position(IGNITE_DFLT_HDR_LEN);
@@ -679,7 +679,7 @@ void CheckCollectionIterators(CollectionType* colType)
     int32_t footerBegin = in.ReadInt32(IGNITE_OFFSET_SCHEMA_OR_RAW_OFF);
     int32_t footerEnd = footerBegin + 5 * 2;
 
-    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_1_BYTE);
+    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_ONE_BYTE);
     BinaryReader reader(&readerImpl);
 
     in.Position(IGNITE_DFLT_HDR_LEN);
@@ -755,7 +755,7 @@ void CheckMapEmpty(MapType* mapType)
     int32_t footerBegin = in.ReadInt32(IGNITE_OFFSET_SCHEMA_OR_RAW_OFF);
     int32_t footerEnd = footerBegin + 5 * 2;
 
-    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_1_BYTE);
+    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_ONE_BYTE);
     BinaryReader reader(&readerImpl);
 
     in.Position(IGNITE_DFLT_HDR_LEN);
@@ -848,7 +848,7 @@ void CheckMap(MapType* mapType)
     int32_t footerBegin = in.ReadInt32(IGNITE_OFFSET_SCHEMA_OR_RAW_OFF);
     int32_t footerEnd = footerBegin + 5 * 2;
 
-    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_1_BYTE);
+    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_ONE_BYTE);
     BinaryReader reader(&readerImpl);
 
     in.Position(IGNITE_DFLT_HDR_LEN);
@@ -1030,7 +1030,7 @@ BOOST_AUTO_TEST_CASE(TestGuidNull)
     int32_t footerBegin = in.ReadInt32(IGNITE_OFFSET_SCHEMA_OR_RAW_OFF);
     int32_t footerEnd = footerBegin + 5;
 
-    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_1_BYTE);
+    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_ONE_BYTE);
     BinaryReader reader(&readerImpl);
     
     in.Position(IGNITE_DFLT_HDR_LEN);
@@ -1115,7 +1115,7 @@ BOOST_AUTO_TEST_CASE(TestString) {
     int32_t footerBegin = in.ReadInt32(IGNITE_OFFSET_SCHEMA_OR_RAW_OFF);
     int32_t footerEnd = footerBegin + 5 * 5;
 
-    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_1_BYTE);
+    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_ONE_BYTE);
     BinaryReader reader(&readerImpl);
 
     in.Position(IGNITE_DFLT_HDR_LEN);
@@ -1193,7 +1193,7 @@ BOOST_AUTO_TEST_CASE(TestStringArrayNull)
     int32_t footerBegin = in.ReadInt32(IGNITE_OFFSET_SCHEMA_OR_RAW_OFF);
     int32_t footerEnd = footerBegin + 5 * 2;
 
-    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_1_BYTE);
+    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_ONE_BYTE);
     BinaryReader reader(&readerImpl);
 
     in.Position(IGNITE_DFLT_HDR_LEN);
@@ -1310,7 +1310,7 @@ BOOST_AUTO_TEST_CASE(TestStringArrayEmpty)
     int32_t footerBegin = in.ReadInt32(IGNITE_OFFSET_SCHEMA_OR_RAW_OFF);
     int32_t footerEnd = footerBegin + 5 * 2;
 
-    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_1_BYTE);
+    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_ONE_BYTE);
     BinaryReader reader(&readerImpl);
 
     in.Position(IGNITE_DFLT_HDR_LEN);
@@ -1437,7 +1437,7 @@ BOOST_AUTO_TEST_CASE(TestStringArray)
     int32_t footerBegin = in.ReadInt32(IGNITE_OFFSET_SCHEMA_OR_RAW_OFF);
     int32_t footerEnd = footerBegin + 5 * 2;
 
-    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_1_BYTE);
+    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_ONE_BYTE);
     BinaryReader reader(&readerImpl);
 
     in.Position(IGNITE_DFLT_HDR_LEN);
@@ -1551,7 +1551,7 @@ BOOST_AUTO_TEST_CASE(TestObject)
     int32_t footerBegin = in.ReadInt32(IGNITE_OFFSET_SCHEMA_OR_RAW_OFF);
     int32_t footerEnd = footerBegin + 5 * 3;
 
-    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_1_BYTE);
+    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_ONE_BYTE);
     BinaryReader reader(&readerImpl);
 
     in.Position(IGNITE_DFLT_HDR_LEN); 
@@ -1595,7 +1595,7 @@ BOOST_AUTO_TEST_CASE(TestNestedObject)
     int32_t footerBegin = in.ReadInt32(IGNITE_OFFSET_SCHEMA_OR_RAW_OFF);
     int32_t footerEnd = footerBegin + 5 * 3;
 
-    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_1_BYTE);
+    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_ONE_BYTE);
     BinaryReader reader(&readerImpl);
 
     in.Position(IGNITE_DFLT_HDR_LEN);
@@ -1637,7 +1637,7 @@ BOOST_AUTO_TEST_CASE(TestArrayNull)
     int32_t footerBegin = in.ReadInt32(IGNITE_OFFSET_SCHEMA_OR_RAW_OFF);
     int32_t footerEnd = footerBegin + 5 * 2;
 
-    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_1_BYTE);
+    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_ONE_BYTE);
     BinaryReader reader(&readerImpl);
 
     in.Position(IGNITE_DFLT_HDR_LEN);
@@ -1713,7 +1713,7 @@ BOOST_AUTO_TEST_CASE(TestArrayEmpty)
     int32_t footerBegin = in.ReadInt32(IGNITE_OFFSET_SCHEMA_OR_RAW_OFF);
     int32_t footerEnd = footerBegin + 5 * 2;
 
-    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_1_BYTE);
+    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_ONE_BYTE);
     BinaryReader reader(&readerImpl);
 
     in.Position(IGNITE_DFLT_HDR_LEN);
@@ -1797,7 +1797,7 @@ BOOST_AUTO_TEST_CASE(TestArray)
     int32_t footerBegin = in.ReadInt32(IGNITE_OFFSET_SCHEMA_OR_RAW_OFF);
     int32_t footerEnd = footerBegin + 5 * 2;
 
-    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_1_BYTE);
+    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_ONE_BYTE);
     BinaryReader reader(&readerImpl);
 
     in.Position(IGNITE_DFLT_HDR_LEN);
@@ -1858,7 +1858,7 @@ BOOST_AUTO_TEST_CASE(TestCollectionNull)
     int32_t footerBegin = in.ReadInt32(IGNITE_OFFSET_SCHEMA_OR_RAW_OFF);
     int32_t footerEnd = footerBegin + 5 * 2;
 
-    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_1_BYTE);
+    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_ONE_BYTE);
     BinaryReader reader(&readerImpl);
 
     in.Position(IGNITE_DFLT_HDR_LEN);
@@ -1944,7 +1944,7 @@ BOOST_AUTO_TEST_CASE(TestMapNull)
     int32_t footerBegin = in.ReadInt32(IGNITE_OFFSET_SCHEMA_OR_RAW_OFF);
     int32_t footerEnd = footerBegin + 5 * 2;
 
-    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_1_BYTE);
+    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_ONE_BYTE);
     BinaryReader reader(&readerImpl);
 
     in.Position(IGNITE_DFLT_HDR_LEN);
@@ -2035,7 +2035,7 @@ BOOST_AUTO_TEST_CASE(TestRawMode)
     int32_t footerBegin = in.ReadInt32(IGNITE_OFFSET_SCHEMA_OR_RAW_OFF);
     int32_t footerEnd = footerBegin;
 
-    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 1000, footerBegin, footerBegin, footerEnd, OFFSET_TYPE_1_BYTE);
+    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 1000, footerBegin, footerBegin, footerEnd, OFFSET_TYPE_ONE_BYTE);
     BinaryReader reader(&readerImpl);
     
     in.Position(IGNITE_DFLT_HDR_LEN);
@@ -2091,10 +2091,10 @@ BOOST_AUTO_TEST_CASE(TestFieldSeek)
     int32_t rawOff;
     int32_t footerBegin;
 
-    if (flags & IGNITE_BINARY_FLAG_RAW_ONLY)
-        footerBegin = len;
-    else
+    if (flags & IGNITE_BINARY_FLAG_HAS_SCHEMA)
         footerBegin = schemaOrRawOff;
+    else
+        footerBegin = len;
 
     int32_t trailingBytes = (len - footerBegin) % 8;
 
@@ -2105,14 +2105,14 @@ BOOST_AUTO_TEST_CASE(TestFieldSeek)
     else
         rawOff = schemaOrRawOff;
 
-    bool usrType = flags & IGNITE_BINARY_FLAG_USER_OBJECT;
+    bool usrType = flags & IGNITE_BINARY_FLAG_USER_TYPE;
 
     footerBegin += pos;
     footerEnd += pos;
 
     BinaryReaderImpl readerImpl(&in, &idRslvr, pos, usrType, 
                                   typeId, hashCode, len, rawOff, 
-                                  footerBegin, footerEnd, OFFSET_TYPE_1_BYTE);
+                                  footerBegin, footerEnd, OFFSET_TYPE_ONE_BYTE);
 
     BinaryReader reader(&readerImpl);
 
@@ -2243,7 +2243,7 @@ BOOST_AUTO_TEST_CASE(TestSchemaOffset2ByteFields)
     int32_t footerBegin = in.ReadInt32(IGNITE_OFFSET_SCHEMA_OR_RAW_OFF);
     int32_t footerEnd = footerBegin + 6 * fieldsNum;
 
-    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_2_BYTE);
+    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_TWO_BYTES);
     BinaryReader reader(&readerImpl);
 
     in.Position(IGNITE_DFLT_HDR_LEN);
@@ -2288,7 +2288,7 @@ BOOST_AUTO_TEST_CASE(TestSchemaOffset4ByteFields)
     int32_t footerBegin = in.ReadInt32(IGNITE_OFFSET_SCHEMA_OR_RAW_OFF);
     int32_t footerEnd = footerBegin + 8 * fieldsNum;
 
-    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_4_BYTE);
+    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_FOUR_BYTES);
     BinaryReader reader(&readerImpl);
 
     in.Position(IGNITE_DFLT_HDR_LEN);
@@ -2328,7 +2328,7 @@ BOOST_AUTO_TEST_CASE(TestSchemaOffset2ByteArray)
     int32_t footerBegin = in.ReadInt32(IGNITE_OFFSET_SCHEMA_OR_RAW_OFF);
     int32_t footerEnd = footerBegin + 6 * 2;
 
-    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_2_BYTE);
+    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_TWO_BYTES);
     BinaryReader reader(&readerImpl);
 
     in.Position(IGNITE_DFLT_HDR_LEN);
@@ -2362,7 +2362,7 @@ BOOST_AUTO_TEST_CASE(TestSchemaOffset4ByteArray)
     int32_t footerBegin = in.ReadInt32(IGNITE_OFFSET_SCHEMA_OR_RAW_OFF);
     int32_t footerEnd = footerBegin + 8 * 2;
 
-    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_4_BYTE);
+    BinaryReaderImpl readerImpl(&in, &idRslvr, 0, true, idRslvr.GetTypeId(), 0, 100, 100, footerBegin, footerEnd, OFFSET_TYPE_FOUR_BYTES);
     BinaryReader reader(&readerImpl);
 
     in.Position(IGNITE_DFLT_HDR_LEN);

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/platforms/cpp/core/include/ignite/impl/binary/binary_common.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/impl/binary/binary_common.h b/modules/platforms/cpp/core/include/ignite/impl/binary/binary_common.h
index d487941..979d2ed 100644
--- a/modules/platforms/cpp/core/include/ignite/impl/binary/binary_common.h
+++ b/modules/platforms/cpp/core/include/ignite/impl/binary/binary_common.h
@@ -21,7 +21,7 @@
 #include <stdint.h>
 
 namespace ignite
-{    
+{
     namespace impl
     {
         namespace binary
@@ -164,19 +164,25 @@ namespace ignite
             /** Read/write map. */
             const int32_t IGNITE_BINARY_MODE_MAP = 3;
 
-            /** User object flag. */
-            const int16_t IGNITE_BINARY_FLAG_USER_OBJECT = 0x0001;
+            /** User type flag. */
+            const int16_t IGNITE_BINARY_FLAG_USER_TYPE = 0x0001;
 
-            /** Raw only flag. */
-            const int16_t IGNITE_BINARY_FLAG_RAW_ONLY = 0x0002;
+            /** Flag: schema exists. */
+            const int16_t IGNITE_BINARY_FLAG_HAS_SCHEMA = 0x0002;
+
+            /** Flag indicating that object has raw data. */
+            const int16_t IGNITE_BINARY_FLAG_HAS_RAW = 0x0004;
 
             /** Flag indicating that schema field offset is one byte long. */
-            const int16_t IGNITE_BINARY_FLAG_OFFSET_1_BYTE = 0x0004;
+            const int16_t IGNITE_BINARY_FLAG_OFFSET_ONE_BYTE = 0x0008;
 
             /** Flag indicating that schema field offset is two byte long. */
-            const int16_t IGNITE_BINARY_FLAG_OFFSET_2_BYTE = 0x0008;
+            const int16_t IGNITE_BINARY_FLAG_OFFSET_TWO_BYTES = 0x0010;
+
+            /** Flag: compact footer, no field IDs. */
+            const int16_t IGNITE_BINARY_FLAG_COMPACT_FOOTER = 0x0020;
         }
-    }    
+    }
 }
 
 #endif
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/platforms/cpp/core/include/ignite/impl/binary/binary_reader_impl.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/impl/binary/binary_reader_impl.h b/modules/platforms/cpp/core/include/ignite/impl/binary/binary_reader_impl.h
index 923553d..0b2b592 100644
--- a/modules/platforms/cpp/core/include/ignite/impl/binary/binary_reader_impl.h
+++ b/modules/platforms/cpp/core/include/ignite/impl/binary/binary_reader_impl.h
@@ -771,6 +771,13 @@ namespace ignite
                             }
 
                             int16_t flags = stream->ReadInt16();
+
+                            if (flags & IGNITE_BINARY_FLAG_COMPACT_FOOTER) {
+                                IGNITE_ERROR_2(ignite::IgniteError::IGNITE_ERR_BINARY,
+                                    "Unsupported binary protocol flag: IGNITE_BINARY_FLAG_COMPACT_FOOTER: ", 
+                                    IGNITE_BINARY_FLAG_COMPACT_FOOTER);
+                            }
+
                             int32_t typeId = stream->ReadInt32();
                             int32_t hashCode = stream->ReadInt32();
                             int32_t len = stream->ReadInt32();
@@ -783,50 +790,43 @@ namespace ignite
                             int32_t rawOff;
                             int32_t footerBegin;
 
-                            if (flags & IGNITE_BINARY_FLAG_RAW_ONLY)
-                                footerBegin = len;
+                            if (flags & IGNITE_BINARY_FLAG_HAS_SCHEMA)
+                                footerBegin = pos + schemaOrRawOff;
                             else
-                                footerBegin = schemaOrRawOff;
+                                footerBegin = pos + len;
 
                             BinaryOffsetType schemaType;
-                            int32_t trailingBytes;
 
-                            if (flags & IGNITE_BINARY_FLAG_OFFSET_1_BYTE)
-                            {
-                                schemaType = OFFSET_TYPE_1_BYTE;
+                            if (flags & IGNITE_BINARY_FLAG_OFFSET_ONE_BYTE)
+                                schemaType = OFFSET_TYPE_ONE_BYTE;
+                            else if (flags & IGNITE_BINARY_FLAG_OFFSET_TWO_BYTES)
+                                schemaType = OFFSET_TYPE_TWO_BYTES;
+                            else
+                                schemaType = OFFSET_TYPE_FOUR_BYTES;
 
-                                trailingBytes = (len - footerBegin) % 5;
-                            }
-                            else if (flags & IGNITE_BINARY_FLAG_OFFSET_2_BYTE)
+                            int32_t footerEnd;
+
+                            if (flags & IGNITE_BINARY_FLAG_HAS_RAW)
                             {
-                                schemaType = OFFSET_TYPE_2_BYTE;
+                                // 4 is the size of RawOffset field at the end of the packet.
+                                footerEnd = pos + len - 4;
 
-                                trailingBytes = (len - footerBegin) % 6;
+                                rawOff = stream->ReadInt32(footerEnd);
                             }
                             else
                             {
-                                schemaType = OFFSET_TYPE_4_BYTE;
-
-                                trailingBytes = (len - footerBegin) % 8;
-                            }
-
-                            int32_t footerEnd = len - trailingBytes;
+                                footerEnd = pos + len;
 
-                            if (trailingBytes)
-                                rawOff = stream->ReadInt32(pos + len - 4);
-                            else
                                 rawOff = schemaOrRawOff;
+                            }
 
-                            bool usrType = flags & IGNITE_BINARY_FLAG_USER_OBJECT;
-
-                            footerBegin += pos;
-                            footerEnd += pos;
+                            bool usrType = flags & IGNITE_BINARY_FLAG_USER_TYPE;
 
                             ignite::binary::BinaryType<T> type;
                             TemplatedBinaryIdResolver<T> idRslvr(type);
                             BinaryReaderImpl readerImpl(stream, &idRslvr, pos, usrType,
-                                                          typeId, hashCode, len, rawOff,
-                                                          footerBegin, footerEnd, schemaType);
+                                                        typeId, hashCode, len, rawOff,
+                                                        footerBegin, footerEnd, schemaType);
                             ignite::binary::BinaryReader reader(&readerImpl);
 
                             T val = type.Read(reader);
@@ -1038,9 +1038,7 @@ namespace ignite
                     {
                         CheckRawMode(false);
                         CheckSingleMode(true);
-
-                        int32_t pos = stream->Position();
-
+                        
                         int32_t fieldId = idRslvr->GetFieldId(typeId, fieldName);
                         int32_t fieldPos = FindField(fieldId);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/platforms/cpp/core/include/ignite/impl/binary/binary_schema.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/impl/binary/binary_schema.h b/modules/platforms/cpp/core/include/ignite/impl/binary/binary_schema.h
index d0a9f26..b100b8c 100644
--- a/modules/platforms/cpp/core/include/ignite/impl/binary/binary_schema.h
+++ b/modules/platforms/cpp/core/include/ignite/impl/binary/binary_schema.h
@@ -39,13 +39,13 @@ namespace ignite
             enum BinaryOffsetType
             {
                 /** Means all field offsets can be fit in one byte. */
-                OFFSET_TYPE_1_BYTE,
+                OFFSET_TYPE_ONE_BYTE,
 
                 /** Means all field offsets can be fit in two bytes. */
-                OFFSET_TYPE_2_BYTE,
+                OFFSET_TYPE_TWO_BYTES,
 
                 /** Means field offsets should be stored in four bytes. */
-                OFFSET_TYPE_4_BYTE
+                OFFSET_TYPE_FOUR_BYTES
             };
 
             /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/platforms/cpp/core/include/ignite/impl/binary/binary_writer_impl.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/impl/binary/binary_writer_impl.h b/modules/platforms/cpp/core/include/ignite/impl/binary/binary_writer_impl.h
index fe31ece..1b47d9e 100644
--- a/modules/platforms/cpp/core/include/ignite/impl/binary/binary_writer_impl.h
+++ b/modules/platforms/cpp/core/include/ignite/impl/binary/binary_writer_impl.h
@@ -601,7 +601,7 @@ namespace ignite
 
                         stream->WriteInt8(IGNITE_HDR_FULL);
                         stream->WriteInt8(IGNITE_PROTO_VER);
-                        stream->WriteInt16(IGNITE_BINARY_FLAG_USER_OBJECT);
+                        stream->WriteInt16(IGNITE_BINARY_FLAG_USER_TYPE);
                         stream->WriteInt32(idRslvr.GetTypeId());
                         stream->WriteInt32(type.GetHashCode(obj));
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/platforms/cpp/core/src/impl/binary/binary_reader_impl.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/src/impl/binary/binary_reader_impl.cpp b/modules/platforms/cpp/core/src/impl/binary/binary_reader_impl.cpp
index e189273..5fcbc75 100644
--- a/modules/platforms/cpp/core/src/impl/binary/binary_reader_impl.cpp
+++ b/modules/platforms/cpp/core/src/impl/binary/binary_reader_impl.cpp
@@ -47,7 +47,7 @@ namespace ignite
             BinaryReaderImpl::BinaryReaderImpl(InteropInputStream* stream) :
                 stream(stream), idRslvr(NULL), pos(0), usrType(false), typeId(0), hashCode(0), len(0),
                 rawOff(0), rawMode(true), elemIdGen(0), elemId(0), elemCnt(-1), elemRead(0), footerBegin(-1),
-                footerEnd(-1), schemaType(OFFSET_TYPE_4_BYTE)
+                footerEnd(-1), schemaType(OFFSET_TYPE_FOUR_BYTES)
             {
                 // No-op.
             }
@@ -637,7 +637,7 @@ namespace ignite
 
                 switch (schemaType)
                 {
-                    case OFFSET_TYPE_1_BYTE:
+                    case OFFSET_TYPE_ONE_BYTE:
                     {
                         for (int32_t schemaPos = footerBegin; schemaPos < footerEnd; schemaPos += 5)
                         {
@@ -649,7 +649,7 @@ namespace ignite
                         break;
                     }
 
-                    case OFFSET_TYPE_2_BYTE:
+                    case OFFSET_TYPE_TWO_BYTES:
                     {
                         for (int32_t schemaPos = footerBegin; schemaPos < footerEnd; schemaPos += 6)
                         {
@@ -661,7 +661,7 @@ namespace ignite
                         break;
                     }
 
-                    case OFFSET_TYPE_4_BYTE:
+                    case OFFSET_TYPE_FOUR_BYTES:
                     {
                         for (int32_t schemaPos = footerBegin; schemaPos < footerEnd; schemaPos += 8)
                         {

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/platforms/cpp/core/src/impl/binary/binary_schema.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/src/impl/binary/binary_schema.cpp b/modules/platforms/cpp/core/src/impl/binary/binary_schema.cpp
index 1596557..0b8025a 100644
--- a/modules/platforms/cpp/core/src/impl/binary/binary_schema.cpp
+++ b/modules/platforms/cpp/core/src/impl/binary/binary_schema.cpp
@@ -70,7 +70,7 @@ namespace ignite
             {
                 switch (GetType())
                 {
-                    case OFFSET_TYPE_1_BYTE:
+                    case OFFSET_TYPE_ONE_BYTE:
                     {
                         for (FieldContainer::const_iterator i = fieldsInfo->begin(); i != fieldsInfo->end(); ++i)
                         {
@@ -80,7 +80,7 @@ namespace ignite
                         break;
                     }
 
-                    case OFFSET_TYPE_2_BYTE:
+                    case OFFSET_TYPE_TWO_BYTES:
                     {
                         for (FieldContainer::const_iterator i = fieldsInfo->begin(); i != fieldsInfo->end(); ++i)
                         {
@@ -90,7 +90,7 @@ namespace ignite
                         break;
                     }
 
-                    case OFFSET_TYPE_4_BYTE:
+                    case OFFSET_TYPE_FOUR_BYTES:
                     {
                         for (FieldContainer::const_iterator i = fieldsInfo->begin(); i != fieldsInfo->end(); ++i)
                         {
@@ -124,11 +124,11 @@ namespace ignite
                 int32_t maxOffset = fieldsInfo->back().offset;
 
                 if (maxOffset < 0x100)
-                    return OFFSET_TYPE_1_BYTE;
+                    return OFFSET_TYPE_ONE_BYTE;
                 else if (maxOffset < 0x10000)
-                    return OFFSET_TYPE_2_BYTE;
+                    return OFFSET_TYPE_TWO_BYTES;
 
-                return OFFSET_TYPE_4_BYTE;
+                return OFFSET_TYPE_FOUR_BYTES;
             }
         }
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/platforms/cpp/core/src/impl/binary/binary_writer_impl.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/src/impl/binary/binary_writer_impl.cpp b/modules/platforms/cpp/core/src/impl/binary/binary_writer_impl.cpp
index 47df19d..d03d6ba 100644
--- a/modules/platforms/cpp/core/src/impl/binary/binary_writer_impl.cpp
+++ b/modules/platforms/cpp/core/src/impl/binary/binary_writer_impl.cpp
@@ -563,11 +563,15 @@ namespace ignite
                 int32_t lenWithoutSchema = stream->Position() - start;
 
                 int32_t nonRawLen = rawPos == -1 ? lenWithoutSchema : rawPos - start;
-                
-                if (schema.Empty())
+
+                uint16_t flags = IGNITE_BINARY_FLAG_USER_TYPE;
+
+                if (rawPos > 0)
+                    flags |= IGNITE_BINARY_FLAG_HAS_RAW;
+
+                if (!HasSchema())
                 {
-                    stream->WriteInt16(start + IGNITE_OFFSET_FLAGS, IGNITE_BINARY_FLAG_USER_OBJECT | 
-                                                                    IGNITE_BINARY_FLAG_RAW_ONLY);
+                    stream->WriteInt16(start + IGNITE_OFFSET_FLAGS, flags);
                     stream->WriteInt32(start + IGNITE_OFFSET_LEN, lenWithoutSchema);
                     stream->WriteInt32(start + IGNITE_OFFSET_SCHEMA_ID, 0);
                     stream->WriteInt32(start + IGNITE_OFFSET_SCHEMA_OR_RAW_OFF, GetRawPosition() - start);
@@ -584,17 +588,14 @@ namespace ignite
 
                     int32_t length = stream->Position() - start;
 
-                    if (schemaType == OFFSET_TYPE_1_BYTE)
-                    {
-                        stream->WriteInt16(start + IGNITE_OFFSET_FLAGS, 
-                            IGNITE_BINARY_FLAG_USER_OBJECT | IGNITE_BINARY_FLAG_OFFSET_1_BYTE);
-                    }
-                    else if (schemaType == OFFSET_TYPE_2_BYTE)
-                    {
-                        stream->WriteInt16(start + IGNITE_OFFSET_FLAGS, 
-                            IGNITE_BINARY_FLAG_USER_OBJECT | IGNITE_BINARY_FLAG_OFFSET_2_BYTE);
-                    }
+                    flags |= IGNITE_BINARY_FLAG_HAS_SCHEMA;
+                    
+                    if (schemaType == OFFSET_TYPE_ONE_BYTE)
+                        flags |= IGNITE_BINARY_FLAG_OFFSET_ONE_BYTE;
+                    else if (schemaType == OFFSET_TYPE_TWO_BYTES)
+                        flags |= IGNITE_BINARY_FLAG_OFFSET_TWO_BYTES;
 
+                    stream->WriteInt16(start + IGNITE_OFFSET_FLAGS, flags);
                     stream->WriteInt32(start + IGNITE_OFFSET_LEN, length);
                     stream->WriteInt32(start + IGNITE_OFFSET_SCHEMA_ID, schemaId);
                     stream->WriteInt32(start + IGNITE_OFFSET_SCHEMA_OR_RAW_OFF, lenWithoutSchema);

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid1.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid1.xml b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid1.xml
index e373b89..9755033 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid1.xml
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid1.xml
@@ -70,6 +70,7 @@
         <!-- Binary marshaller configuration -->
         <property name="marshaller">
             <bean class="org.apache.ignite.marshaller.portable.PortableMarshaller">
+                <property name="compactFooter" value="false" />
                 <property name="typeConfigurations">
                     <list>
                         <bean class="org.apache.ignite.binary.BinaryTypeConfiguration">

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/marshaller-explicit.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/marshaller-explicit.xml b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/marshaller-explicit.xml
index 11b87bd..74444a1 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/marshaller-explicit.xml
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/marshaller-explicit.xml
@@ -26,7 +26,9 @@
         <property name="connectorConfiguration"><null/></property>
 
         <property name="marshaller">
-            <bean class="org.apache.ignite.marshaller.portable.PortableMarshaller" />
+            <bean class="org.apache.ignite.marshaller.portable.PortableMarshaller">
+                <property name="compactFooter" value="false"/>
+            </bean>
         </property>
 
         <property name="discoverySpi">

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectBuilder.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectBuilder.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectBuilder.cs
index 1840ab9..97cc381 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectBuilder.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectBuilder.cs
@@ -678,28 +678,37 @@ namespace Apache.Ignite.Core.Impl.Binary
                                 WriteField(ctx, valEntry.Value);
                             }
 
+                            var flags = inHeader.IsUserType
+                                ? BinaryObjectHeader.Flag.UserType
+                                : BinaryObjectHeader.Flag.None;
+
                             // Write raw data.
                             int outRawOff = outStream.Position - outStartPos;
 
-                            int inRawOff = inHeader.GetRawOffset(inStream, inStartPos);
-                            int inRawLen = inHeader.SchemaOffset - inRawOff;
+                            if (inHeader.HasRaw)
+                            {
+                                var inRawOff = inHeader.GetRawOffset(inStream, inStartPos);
+                                var inRawLen = inHeader.SchemaOffset - inRawOff;
+
+                                flags |= BinaryObjectHeader.Flag.HasRaw;
 
-                            if (inRawLen > 0)
                                 outStream.Write(inStream.InternalArray, inStartPos + inRawOff, inRawLen);
+                            }
 
                             // Write schema
                             int outSchemaOff = outRawOff;
                             var schemaPos = outStream.Position;
                             int outSchemaId;
-                            short flags;
 
-                            var hasSchema = outSchema.WriteSchema(outStream, schemaIdx, out outSchemaId, out flags);
+                            var hasSchema = outSchema.WriteSchema(outStream, schemaIdx, out outSchemaId, ref flags);
 
                             if (hasSchema)
                             {
                                 outSchemaOff = schemaPos - outStartPos;
+                                
+                                flags |= BinaryObjectHeader.Flag.HasSchema;
 
-                                if (inRawLen > 0)
+                                if (inHeader.HasRaw)
                                     outStream.WriteInt(outRawOff);
                             }
 
@@ -707,8 +716,8 @@ namespace Apache.Ignite.Core.Impl.Binary
 
                             var outHash = changeHash ? hash : inHeader.HashCode;
 
-                            var outHeader = new BinaryObjectHeader(inHeader.IsUserType, inHeader.TypeId, outHash,
-                                outLen, outSchemaId, outSchemaOff, !hasSchema, flags);
+                            var outHeader = new BinaryObjectHeader(inHeader.TypeId, outHash, outLen, 
+                                outSchemaId, outSchemaOff, flags);
 
                             BinaryObjectHeader.Write(outHeader, outStream, outStartPos);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectHeader.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectHeader.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectHeader.cs
index 59cb29c1..573e014 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectHeader.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectHeader.cs
@@ -33,22 +33,38 @@ namespace Apache.Ignite.Core.Impl.Binary
         /** Size, equals to sizeof(BinaryObjectHeader). */
         public const int Size = 24;
 
-        /** User type flag. */
-        public const short FlagUserType = 0x1;
+        /// <summary>
+        /// Flags.
+        /// </summary>
+        [Flags]
+        public enum Flag : short
+        {
+            /** No flags. */
+            None            = 0x00,
+
+            /** Flag: user type. */
+            UserType        = 0x01,
+            
+            /** Flag: schema exists. */
+            HasSchema       = 0x02,
+
+            /** Flag indicating that object has raw data. */
+            HasRaw          = 0x04,
 
-        /** Raw only flag. */
-        public const short FlagRawOnly = 0x2;
+            /** Flag: offsets take 1 byte. */
+            OffsetOneByte   = 0x08,
 
-        /** Byte-sized field offsets flag. */
-        public const short FlagByteOffsets = 0x4;
+            /** Flag: offsets take 2 bytes. */
+            OffsetTwoBytes  = 0x10,
 
-        /** Short-sized field offsets flag. */
-        public const short FlagShortOffsets = 0x8;
+            /** Flag: compact footer, no field IDs. */
+            CompactFooter   = 0x20
+        }
 
         /** Actual header layout */
         public readonly byte Header;        // Header code, always 103 (HdrFull)
         public readonly byte Version;       // Protocol version
-        public readonly short Flags;        // Flags
+        public readonly Flag Flags;         // Flags
         public readonly int TypeId;         // Type ID
         public readonly int HashCode;       // Hash code
         public readonly int Length;         // Length, including header
@@ -58,16 +74,13 @@ namespace Apache.Ignite.Core.Impl.Binary
         /// <summary>
         /// Initializes a new instance of the <see cref="BinaryObjectHeader" /> struct.
         /// </summary>
-        /// <param name="userType">User type flag.</param>
         /// <param name="typeId">Type ID.</param>
         /// <param name="hashCode">Hash code.</param>
         /// <param name="length">Length.</param>
         /// <param name="schemaId">Schema ID.</param>
         /// <param name="schemaOffset">Schema offset.</param>
-        /// <param name="rawOnly">Raw flag.</param>
         /// <param name="flags">The flags.</param>
-        public BinaryObjectHeader(bool userType, int typeId, int hashCode, int length, int schemaId, int schemaOffset, 
-            bool rawOnly, short flags)
+        public BinaryObjectHeader(int typeId, int hashCode, int length, int schemaId, int schemaOffset, Flag flags)
         {
             Header = BinaryUtils.HdrFull;
             Version = BinaryUtils.ProtoVer;
@@ -75,12 +88,6 @@ namespace Apache.Ignite.Core.Impl.Binary
             Debug.Assert(schemaOffset <= length);
             Debug.Assert(schemaOffset >= Size);
 
-            if (userType)
-                flags |= FlagUserType;
-
-            if (rawOnly)
-                flags |= FlagRawOnly;
-
             Flags = flags;
 
             TypeId = typeId;
@@ -98,7 +105,7 @@ namespace Apache.Ignite.Core.Impl.Binary
         {
             Header = stream.ReadByte();
             Version = stream.ReadByte();
-            Flags = stream.ReadShort();
+            Flags = (Flag) stream.ReadShort();
             Length = stream.ReadInt();
             TypeId = stream.ReadInt();
             HashCode = stream.ReadInt();
@@ -114,7 +121,7 @@ namespace Apache.Ignite.Core.Impl.Binary
         {
             stream.WriteByte(Header);
             stream.WriteByte(Version);
-            stream.WriteShort(Flags);
+            stream.WriteShort((short) Flags);
             stream.WriteInt(Length);
             stream.WriteInt(TypeId);
             stream.WriteInt(HashCode);
@@ -123,31 +130,35 @@ namespace Apache.Ignite.Core.Impl.Binary
         }
 
         /// <summary>
-        /// Gets a user type flag.
+        /// Gets the user type flag.
         /// </summary>
         public bool IsUserType
         {
-            get { return (Flags & FlagUserType) == FlagUserType; }
+            get { return (Flags & Flag.UserType) == Flag.UserType; }
         }
 
         /// <summary>
-        /// Gets a raw-only flag.
+        /// Gets the schema flag.
         /// </summary>
-        public bool IsRawOnly
+        public bool HasSchema
         {
-            get { return (Flags & FlagRawOnly) == FlagRawOnly; }
+            get { return (Flags & Flag.HasSchema) == Flag.HasSchema; }
         }
 
         /// <summary>
-        /// Gets a value indicating whether this instance has raw offset.
+        /// Gets the raw flag.
         /// </summary>
-        public bool HasRawOffset
+        public bool HasRaw
         {
-            get
-            {
-                // Remainder => raw offset is the very last 4 bytes in object.
-                return !IsRawOnly && ((Length - SchemaOffset) % SchemaFieldSize) == 4;
-            }
+            get { return (Flags & Flag.HasRaw) == Flag.HasRaw; }
+        }
+
+        /// <summary>
+        /// Gets the compact footer flag.
+        /// </summary>
+        public bool IsCompactFooter
+        {
+            get { return (Flags & Flag.CompactFooter) == Flag.CompactFooter; }
         }
 
         /// <summary>
@@ -157,10 +168,10 @@ namespace Apache.Ignite.Core.Impl.Binary
         {
             get
             {
-                if ((Flags & FlagByteOffsets) == FlagByteOffsets)
+                if ((Flags & Flag.OffsetOneByte) == Flag.OffsetOneByte)
                     return 1;
 
-                if ((Flags & FlagShortOffsets) == FlagShortOffsets)
+                if ((Flags & Flag.OffsetTwoBytes) == Flag.OffsetTwoBytes)
                     return 2;
 
                 return 4;
@@ -182,7 +193,7 @@ namespace Apache.Ignite.Core.Impl.Binary
         {
             get
             {
-                if (IsRawOnly)
+                if (!HasSchema)
                     return 0;
 
                 var schemaSize = Length - SchemaOffset;
@@ -201,7 +212,7 @@ namespace Apache.Ignite.Core.Impl.Binary
         {
             Debug.Assert(stream != null);
 
-            if (!HasRawOffset)
+            if (!HasRaw || !HasSchema)
                 return SchemaOffset;
 
             stream.Seek(position + Length - 4, SeekOrigin.Begin);
@@ -219,6 +230,8 @@ namespace Apache.Ignite.Core.Impl.Binary
         {
             Debug.Assert(stream != null);
 
+            ThrowIfUnsupported();
+
             var schemaSize = SchemaFieldCount;
 
             if (schemaSize == 0)
@@ -259,6 +272,8 @@ namespace Apache.Ignite.Core.Impl.Binary
         {
             Debug.Assert(stream != null);
 
+            ThrowIfUnsupported();
+
             var schemaSize = SchemaFieldCount;
 
             if (schemaSize == 0)
@@ -297,10 +312,10 @@ namespace Apache.Ignite.Core.Impl.Binary
         /// <param name="offset">Offset in the array.</param>
         /// <param name="count">Field count to write.</param>
         /// <returns>
-        /// Flags according to offset sizes: <see cref="BinaryObjectHeader.FlagByteOffsets" />,
-        /// <see cref="BinaryObjectHeader.FlagShortOffsets" />, or 0.
+        /// Flags according to offset sizes: <see cref="Flag.OffsetOneByte" />,
+        /// <see cref="Flag.OffsetTwoBytes" />, or 0.
         /// </returns>
-        public static unsafe short WriteSchema(BinaryObjectSchemaField[] fields, IBinaryStream stream, int offset,
+        public static unsafe Flag WriteSchema(BinaryObjectSchemaField[] fields, IBinaryStream stream, int offset,
             int count)
         {
             Debug.Assert(fields != null);
@@ -324,7 +339,7 @@ namespace Apache.Ignite.Core.Impl.Binary
                         stream.WriteByte((byte)field.Offset);
                     }
 
-                    return FlagByteOffsets;
+                    return Flag.OffsetOneByte;
                 }
 
                 if (maxFieldOffset <= ushort.MaxValue)
@@ -338,7 +353,7 @@ namespace Apache.Ignite.Core.Impl.Binary
                         stream.WriteShort((short)field.Offset);
                     }
 
-                    return FlagShortOffsets;
+                    return Flag.OffsetTwoBytes;
                 }
 
                 if (BitConverter.IsLittleEndian)
@@ -359,9 +374,8 @@ namespace Apache.Ignite.Core.Impl.Binary
                     }
                 }
 
-                return 0;
+                return Flag.None;
             }
-
         }
 
         /// <summary>
@@ -396,24 +410,27 @@ namespace Apache.Ignite.Core.Impl.Binary
 
             stream.Seek(position, SeekOrigin.Begin);
 
+            BinaryObjectHeader hdr;
+
             if (BitConverter.IsLittleEndian)
             {
-                var hdr = new BinaryObjectHeader();
-
                 stream.Read((byte*) &hdr, Size);
 
                 Debug.Assert(hdr.Version == BinaryUtils.ProtoVer);
                 Debug.Assert(hdr.SchemaOffset <= hdr.Length);
                 Debug.Assert(hdr.SchemaOffset >= Size);
 
-                // Only one of the flags can be set
-                var f = hdr.Flags;
-                Debug.Assert((f & (FlagShortOffsets | FlagByteOffsets)) != (FlagShortOffsets | FlagByteOffsets));
-
-                return hdr;
             }
+            else
+                hdr = new BinaryObjectHeader(stream);
+
+            hdr.ThrowIfUnsupported();
 
-            return new BinaryObjectHeader(stream);
+            // Only one of the flags can be set
+            var f = hdr.Flags;
+            Debug.Assert((f & (Flag.OffsetOneByte | Flag.OffsetTwoBytes)) !=
+                         (Flag.OffsetOneByte | Flag.OffsetTwoBytes));
+            return hdr;
         }
 
         /** <inheritdoc> */
@@ -465,5 +482,15 @@ namespace Apache.Ignite.Core.Impl.Binary
         {
             return !left.Equals(right);
         }
+
+        /// <summary>
+        /// Throws an exception if current header represents unsupported mode.
+        /// </summary>
+        private void ThrowIfUnsupported()
+        {
+            // Compact schema is not supported
+            if (IsCompactFooter)
+                throw new NotSupportedException("Compact binary object footer is not supported in Ignite.NET.");
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchemaHolder.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchemaHolder.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchemaHolder.cs
index 75ff2c5..65b6fc0 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchemaHolder.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchemaHolder.cs
@@ -82,22 +82,21 @@ namespace Apache.Ignite.Core.Impl.Binary
         /// <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="BinaryObjectHeader.FlagByteOffsets" />,
-        /// <see cref="BinaryObjectHeader.FlagShortOffsets" />, or 0.</param>
+        /// <param name="flags">Flags according to offset sizes.</param>
         /// <returns>
         /// True if current schema was non empty; false otherwise.
         /// </returns>
-        public bool WriteSchema(IBinaryStream stream, int schemaOffset, out int schemaId, out short flags)
+        public bool WriteSchema(IBinaryStream stream, int schemaOffset, out int schemaId, 
+            ref BinaryObjectHeader.Flag flags)
         {
             schemaId = Fnv1Hash.Basis;
-            flags = 0;
 
             var count = _idx - schemaOffset;
 
             if (count == 0) 
                 return false;
 
-            flags = BinaryObjectHeader.WriteSchema(_fields, stream, schemaOffset, count);
+            flags |= BinaryObjectHeader.WriteSchema(_fields, stream, schemaOffset, count);
 
             for (var i = schemaOffset; i < _idx; i++)
                 schemaId = Fnv1Hash.Update(schemaId, _fields[i].Id);

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReader.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReader.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReader.cs
index 53f6f4a..9aeb908 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReader.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReader.cs
@@ -898,7 +898,7 @@ namespace Apache.Ignite.Core.Impl.Binary
             if (_curRaw)
                 throw new BinaryObjectException("Cannot read named fields after raw data is read.");
 
-            if (_curHdr.IsRawOnly)
+            if (!_curHdr.HasSchema)
                 return false;
 
             var actionId = _curStruct.CurStructAction;

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryWriter.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryWriter.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryWriter.cs
index 58973f7..e09a7f4 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryWriter.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryWriter.cs
@@ -1090,20 +1090,31 @@ namespace Apache.Ignite.Core.Impl.Binary
                     var schemaOffset = _stream.Position - pos;
 
                     int schemaId;
-                    short flags;
-                    var hasSchema = _schema.WriteSchema(_stream, schemaIdx, out schemaId, out flags);
+                    
+                    var flags = desc.UserType
+                        ? BinaryObjectHeader.Flag.UserType
+                        : BinaryObjectHeader.Flag.None;
 
-                    if (!hasSchema)
+                    var hasSchema = _schema.WriteSchema(_stream, schemaIdx, out schemaId, ref flags);
+
+                    if (hasSchema)
+                    {
+                        flags |= BinaryObjectHeader.Flag.HasSchema;
+
+                        // Calculate and write header.
+                        if (_curRawPos > 0)
+                            _stream.WriteInt(_curRawPos - pos); // raw offset is in the last 4 bytes
+                    }
+                    else
                         schemaOffset = BinaryObjectHeader.Size;
 
-                    // Calculate and write header.
-                    if (hasSchema && _curRawPos > 0)
-                        _stream.WriteInt(_curRawPos - pos); // raw offset is in the last 4 bytes
+                    if (_curRawPos > 0)
+                        flags |= BinaryObjectHeader.Flag.HasRaw;
 
                     var len = _stream.Position - pos;
 
-                    var header = new BinaryObjectHeader(desc.UserType, desc.TypeId, obj.GetHashCode(), len,
-                        schemaId, schemaOffset, !hasSchema, flags);
+                    var header = new BinaryObjectHeader(desc.TypeId, obj.GetHashCode(), len,
+                        schemaId, schemaOffset, flags);
 
                     BinaryObjectHeader.Write(header, _stream, pos);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4a8f83/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/IgniteBinary.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/IgniteBinary.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/IgniteBinary.cs
index 927ebaf..ecc6807 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/IgniteBinary.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/IgniteBinary.cs
@@ -166,7 +166,8 @@ namespace Apache.Ignite.Core.Impl.Binary
         {
             var len = BinaryObjectHeader.Size;
 
-            var hdr = new BinaryObjectHeader(desc.UserType, desc.TypeId, 0, len, 0, len, true, 0);
+            var hdr = new BinaryObjectHeader(desc.TypeId, 0, len, 0, len,
+                desc.UserType ? BinaryObjectHeader.Flag.UserType : BinaryObjectHeader.Flag.None);
 
             var stream = new BinaryHeapStream(len);