You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by nt...@apache.org on 2016/03/15 19:17:58 UTC

[02/35] ignite git commit: IGNITE-2702: .NET: Implemented compact footers optimization for binary serialization. This closes #523.

IGNITE-2702: .NET: Implemented compact footers optimization for binary serialization. This closes #523.


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

Branch: refs/heads/ignite-2791
Commit: c6c93892f13c55fb6e48e542970e30b26d014511
Parents: 1f328e4
Author: Pavel Tupitsyn <pt...@gridgain.com>
Authored: Wed Mar 9 17:15:19 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Wed Mar 9 17:15:19 2016 +0300

----------------------------------------------------------------------
 .../ignite/internal/binary/BinarySchema.java    |  11 +-
 .../processors/platform/PlatformContext.java    |  11 +-
 .../platform/PlatformContextImpl.java           |  72 ++++-
 .../platform/cluster/PlatformClusterGroup.java  |  12 +
 .../PlatformDotNetConfigurationClosure.java     |   5 -
 .../utils/PlatformConfigurationUtils.java       |  14 +
 .../ignite/platform/PlatformSqlQueryTask.java   | 117 +++++++++
 .../Apache.Ignite.Core.Tests.csproj             |   8 +-
 .../Binary/BinaryBuilderSelfTest.cs             |  33 ++-
 .../Binary/BinaryBuilderSelfTestFullFooter.cs   |  31 +++
 .../Binary/BinaryCompactFooterInteropTest.cs    | 129 +++++++++
 .../Binary/BinarySelfTest.cs                    |  23 +-
 .../Binary/BinarySelfTestFullFooter.cs          |  35 +++
 .../Compute/ComputeApiTest.cs                   |  57 +++-
 .../Compute/ComputeApiTestFullFooter.cs         |  65 +++++
 .../Config/Compute/compute-grid1.xml            |  20 +-
 .../Config/Compute/compute-grid2.xml            |   2 +-
 .../Config/Compute/compute-grid3.xml            |   2 +-
 .../Apache.Ignite.Core.Tests/Config/binary.xml  |  56 ----
 .../IgniteConfigurationSerializerTest.cs        |   3 +-
 .../IgniteConfigurationTest.cs                  |   2 +
 .../Services/ServicesTest.cs                    |  26 +-
 .../Services/ServicesTestFullFooter.cs          |  33 +++
 .../Apache.Ignite.Core.csproj                   |   1 +
 .../Binary/BinaryConfiguration.cs               |  44 +++-
 .../Apache.Ignite.Core/IgniteConfiguration.cs   |  15 ++
 .../IgniteConfigurationSection.xsd              |   1 +
 .../Apache.Ignite.Core/Impl/Binary/Binary.cs    |   8 +-
 .../Impl/Binary/BinaryObject.cs                 |   9 +-
 .../Impl/Binary/BinaryObjectBuilder.cs          |   9 +-
 .../Impl/Binary/BinaryObjectHeader.cs           | 176 +------------
 .../Impl/Binary/BinaryObjectSchema.cs           |  20 ++
 .../Impl/Binary/BinaryObjectSchemaHolder.cs     |  18 +-
 .../Impl/Binary/BinaryObjectSchemaSerializer.cs | 262 +++++++++++++++++++
 .../Impl/Binary/BinaryReader.cs                 |  46 +++-
 .../Impl/Binary/BinaryWriter.cs                 |  20 +-
 .../Impl/Binary/Marshaller.cs                   |  12 +-
 .../Impl/Binary/Metadata/BinaryType.cs          |  28 +-
 .../Impl/Cluster/ClusterGroupImpl.cs            |  15 ++
 .../dotnet/Apache.Ignite.Core/Impl/Ignite.cs    |  19 ++
 .../Apache.Ignite.Core/Impl/PlatformTarget.cs   |  47 +++-
 41 files changed, 1204 insertions(+), 313 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/core/src/main/java/org/apache/ignite/internal/binary/BinarySchema.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinarySchema.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinarySchema.java
index 04124e0..125719e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinarySchema.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinarySchema.java
@@ -86,7 +86,7 @@ public class BinarySchema implements Externalizable {
      * @param schemaId Schema ID.
      * @param fieldIds Field IDs.
      */
-    private BinarySchema(int schemaId, List<Integer> fieldIds) {
+    public BinarySchema(int schemaId, List<Integer> fieldIds) {
         assert fieldIds != null;
 
         this.schemaId = schemaId;
@@ -261,6 +261,15 @@ public class BinarySchema implements Externalizable {
     }
 
     /**
+     * Gets field ids array.
+     *
+     * @return Field ids.
+     */
+    public int[] fieldIds() {
+        return ids;
+    }
+
+    /**
      * Parse values.
      *
      * @param vals Values.

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContext.java
index e88d57b..da92c6c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContext.java
@@ -143,6 +143,15 @@ public interface PlatformContext {
     public void writeAllMetadata(BinaryRawWriterEx writer);
 
     /**
+     * Write schema for the given type ID and schema ID.
+     *
+     * @param writer Writer.
+     * @param typeId Type ID.
+     * @param schemaId Schema ID.
+     */
+    public void writeSchema(BinaryRawWriterEx writer, int typeId, int schemaId);
+
+    /**
      * Write cluster metrics.
      *
      * @param writer Writer.
@@ -279,4 +288,4 @@ public interface PlatformContext {
      * @return Current platform name.
      */
     public String platform();
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java
index b45414a..d89176b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java
@@ -34,9 +34,13 @@ import org.apache.ignite.events.JobEvent;
 import org.apache.ignite.events.SwapSpaceEvent;
 import org.apache.ignite.events.TaskEvent;
 import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.binary.BinaryContext;
+import org.apache.ignite.internal.binary.BinaryMetadata;
 import org.apache.ignite.internal.binary.BinaryRawReaderEx;
 import org.apache.ignite.internal.binary.BinaryRawWriterEx;
 import org.apache.ignite.internal.binary.BinaryReaderExImpl;
+import org.apache.ignite.internal.binary.BinarySchema;
+import org.apache.ignite.internal.binary.BinarySchemaRegistry;
 import org.apache.ignite.internal.binary.BinaryTypeImpl;
 import org.apache.ignite.internal.binary.GridBinaryMarshaller;
 import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl;
@@ -73,11 +77,13 @@ import org.apache.ignite.lang.IgniteBiTuple;
 import org.jetbrains.annotations.Nullable;
 
 import java.sql.Timestamp;
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.UUID;
@@ -86,6 +92,7 @@ import java.util.concurrent.ConcurrentHashMap;
 /**
  * Implementation of platform context.
  */
+@SuppressWarnings("TypeMayBeWeakened")
 public class PlatformContextImpl implements PlatformContext {
     /** Supported event types. */
     private static final Set<Integer> evtTyps;
@@ -365,13 +372,36 @@ public class PlatformContextImpl implements PlatformContext {
 
                     boolean isEnum = reader.readBoolean();
 
-                    return new Metadata(typeId, typeName, affKey, fields, isEnum);
+                    // Read schemas
+                    int schemaCnt = reader.readInt();
+
+                    List<BinarySchema> schemas = null;
+
+                    if (schemaCnt > 0) {
+                        schemas = new ArrayList<>(schemaCnt);
+
+                        for (int i = 0; i < schemaCnt; i++) {
+                            int id = reader.readInt();
+                            int fieldCnt = reader.readInt();
+                            List<Integer> fieldIds = new ArrayList<>(fieldCnt);
+
+                            for (int j = 0; j < fieldCnt; j++)
+                                fieldIds.add(reader.readInt());
+
+                            schemas.add(new BinarySchema(id, fieldIds));
+                        }
+                    }
+
+                    return new Metadata(typeId, typeName, affKey, fields, isEnum, schemas);
                 }
             }
         );
 
+        BinaryContext binCtx = cacheObjProc.binaryContext();
+
         for (Metadata meta : metas)
-            cacheObjProc.updateMetadata(meta.typeId, meta.typeName, meta.affKey, meta.fields, meta.isEnum);
+            binCtx.updateMetadata(meta.typeId, new BinaryMetadata(meta.typeId,
+                meta.typeName, meta.fields, meta.affKey, meta.schemas, meta.isEnum));
     }
 
     /** {@inheritDoc} */
@@ -389,6 +419,30 @@ public class PlatformContextImpl implements PlatformContext {
             writeMetadata0(writer, cacheObjProc.typeId(m.typeName()), m);
     }
 
+    /** {@inheritDoc} */
+    @Override public void writeSchema(BinaryRawWriterEx writer, int typeId, int schemaId) {
+        BinarySchemaRegistry schemaReg = cacheObjProc.binaryContext().schemaRegistry(typeId);
+        BinarySchema schema = schemaReg.schema(schemaId);
+
+        if (schema == null) {
+            BinaryTypeImpl meta = (BinaryTypeImpl)cacheObjProc.metadata(typeId);
+
+            for (BinarySchema typeSchema : meta.metadata().schemas()) {
+                if (schemaId == typeSchema.schemaId()) {
+                    schema = typeSchema;
+                    break;
+                }
+            }
+
+            if (schema != null)
+                schemaReg.addSchema(schemaId, schema);
+        }
+
+        int[] fieldIds = schema == null ? null : schema.fieldIds();
+
+        writer.writeIntArray(fieldIds);
+    }
+
     /**
      * Write binary metadata.
      *
@@ -402,7 +456,8 @@ public class PlatformContextImpl implements PlatformContext {
         else {
             writer.writeBoolean(true);
 
-            Map<String, Integer> fields = ((BinaryTypeImpl)meta).metadata().fieldsMap();
+            BinaryMetadata meta0 = ((BinaryTypeImpl) meta).metadata();
+            Map<String, Integer> fields = meta0.fieldsMap();
 
             writer.writeInt(typeId);
             writer.writeString(meta.typeName());
@@ -651,21 +706,26 @@ public class PlatformContextImpl implements PlatformContext {
         /** Enum flag. */
         private final boolean isEnum;
 
+        /** Schemas. */
+        private final List<BinarySchema> schemas;
+
         /**
          * Constructor.
-         *
-         * @param typeId Type ID.
+         *  @param typeId Type ID.
          * @param typeName Type name.
          * @param affKey Affinity key.
          * @param fields Fields.
          * @param isEnum Enum flag.
+         * @param schemas Schemas.
          */
-        public Metadata(int typeId, String typeName, String affKey, Map<String, Integer> fields, boolean isEnum) {
+        private Metadata(int typeId, String typeName, String affKey, Map<String, Integer> fields, boolean isEnum,
+            List<BinarySchema> schemas) {
             this.typeId = typeId;
             this.typeName = typeName;
             this.affKey = affKey;
             this.fields = fields;
             this.isEnum = isEnum;
+            this.schemas = schemas;
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java
index f60766b..d80079c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java
@@ -77,6 +77,9 @@ public class PlatformClusterGroup extends PlatformAbstractTarget {
     /** */
     private static final int OP_TOPOLOGY = 14;
 
+    /** */
+    private static final int OP_SCHEMA = 15;
+
     /** Projection. */
     private final ClusterGroupEx prj;
 
@@ -188,6 +191,15 @@ public class PlatformClusterGroup extends PlatformAbstractTarget {
                 break;
             }
 
+            case OP_SCHEMA: {
+                int typeId = reader.readInt();
+                int schemaId = reader.readInt();
+
+                platformCtx.writeSchema(writer, typeId, schemaId);
+
+                break;
+            }
+
             default:
                 super.processInStreamOutStream(type, reader, writer);
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java
index 8728d77..db2fa4d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java
@@ -109,7 +109,6 @@ public class PlatformDotNetConfigurationClosure extends PlatformAbstractConfigur
         if (bCfg == null) {
             bCfg = new BinaryConfiguration();
 
-            bCfg.setCompactFooter(false);
             bCfg.setNameMapper(new BinaryBasicNameMapper(true));
             bCfg.setIdMapper(new BinaryBasicIdMapper(true));
 
@@ -142,10 +141,6 @@ public class PlatformDotNetConfigurationClosure extends PlatformAbstractConfigur
             }
         }
 
-        if (bCfg.isCompactFooter())
-            throw new IgniteException("Unsupported " + BinaryMarshaller.class.getName() +
-                " \"compactFooter\" flag: must be false when running Apache Ignite.NET.");
-
         // Set Ignite home so that marshaller context works.
         String ggHome = igniteCfg.getIgniteHome();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
index c0e9f1b..50728a1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
@@ -28,6 +28,7 @@ import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.cache.QueryEntity;
 import org.apache.ignite.cache.QueryIndex;
 import org.apache.ignite.cache.QueryIndexType;
+import org.apache.ignite.configuration.BinaryConfiguration;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.binary.*;
@@ -259,6 +260,13 @@ import java.util.Map;
 
         readCacheConfigurations(in, cfg);
         readDiscoveryConfiguration(in, cfg);
+
+        if (in.readBoolean()) {
+            if (cfg.getBinaryConfiguration() == null)
+                cfg.setBinaryConfiguration(new BinaryConfiguration());
+
+            cfg.getBinaryConfiguration().setCompactFooter(in.readBoolean());
+        }
     }
 
     /**
@@ -544,6 +552,12 @@ import java.util.Map;
 
         writeDiscoveryConfiguration(w, cfg.getDiscoverySpi());
 
+        BinaryConfiguration bc = cfg.getBinaryConfiguration();
+        w.writeBoolean(bc != null);
+
+        if (bc != null)
+            w.writeBoolean(bc.isCompactFooter());
+
         w.writeString(cfg.getIgniteHome());
 
         w.writeLong(ManagementFactory.getMemoryMXBean().getHeapMemoryUsage().getInit());

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/core/src/test/java/org/apache/ignite/platform/PlatformSqlQueryTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/platform/PlatformSqlQueryTask.java b/modules/core/src/test/java/org/apache/ignite/platform/PlatformSqlQueryTask.java
new file mode 100644
index 0000000..41d83aa
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/platform/PlatformSqlQueryTask.java
@@ -0,0 +1,117 @@
+/*
+ * 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.platform;
+
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.query.SqlQuery;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.compute.ComputeJob;
+import org.apache.ignite.compute.ComputeJobAdapter;
+import org.apache.ignite.compute.ComputeJobResult;
+import org.apache.ignite.compute.ComputeTaskAdapter;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.resources.IgniteInstanceResource;
+import org.jetbrains.annotations.Nullable;
+
+import javax.cache.Cache;
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Task that performs an SQL query and returns results.
+ */
+public class PlatformSqlQueryTask extends ComputeTaskAdapter<String, Object> {
+    /** {@inheritDoc} */
+    @Nullable @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid,
+        @Nullable String arg) throws IgniteException {
+        return Collections.singletonMap(new SqlQueryJob(arg), F.first(subgrid));
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public Object reduce(List<ComputeJobResult> results) throws IgniteException {
+        ComputeJobResult res = results.get(0);
+
+        if (res.getException() != null)
+            throw res.getException();
+        else
+            return results.get(0).getData();
+    }
+
+    /**
+     * Job.
+     */
+    private static class SqlQueryJob extends ComputeJobAdapter implements Externalizable {
+        /** */
+        @IgniteInstanceResource
+        private Ignite ignite;
+
+        /** Argument. */
+        private String arg;
+
+        /**
+         * Constructor.
+         */
+        public SqlQueryJob() {
+            // No-op.
+        }
+
+        /**
+         * Constructor.
+         *
+         * @param arg Argument.
+         */
+        private SqlQueryJob(String arg) {
+            this.arg = arg;
+        }
+
+        /** {@inheritDoc} */
+        @Nullable @Override public Object execute() {
+            IgniteCache<Integer, PlatformComputeBinarizable> cache = ignite.cache(null);
+
+            SqlQuery<Integer, PlatformComputeBinarizable> qry = new SqlQuery<>("PlatformComputeBinarizable", arg);
+
+            List<Cache.Entry<Integer, PlatformComputeBinarizable>> qryRes = cache.query(qry).getAll();
+
+            Collection<PlatformComputeBinarizable> res = new ArrayList<>(qryRes.size());
+
+            for (Cache.Entry<Integer, PlatformComputeBinarizable> e : qryRes)
+                res.add(e.getValue());
+
+            return res;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void writeExternal(ObjectOutput out) throws IOException {
+            out.writeObject(arg);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+            arg = (String)in.readObject();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
index bb56a3d..8c266d7 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
@@ -61,6 +61,9 @@
     <Reference Include="System.XML" />
   </ItemGroup>
   <ItemGroup>
+    <Compile Include="Binary\BinaryBuilderSelfTestFullFooter.cs" />
+    <Compile Include="Binary\BinaryCompactFooterInteropTest.cs" />
+    <Compile Include="Binary\BinarySelfTestFullFooter.cs" />
     <Compile Include="Cache\CacheAffinityFieldTest.cs" />
     <Compile Include="Cache\CacheConfigurationTest.cs" />
     <Compile Include="Cache\CacheDynamicStartTest.cs" />
@@ -91,6 +94,7 @@
     <Compile Include="Cache\Store\CacheTestParallelLoadStore.cs" />
     <Compile Include="Cache\Store\CacheTestStore.cs" />
     <Compile Include="Compute\CancellationTest.cs" />
+    <Compile Include="Compute\ComputeApiTestFullFooter.cs" />
     <Compile Include="Compute\Forked\ForkedBinarizableClosureTaskTest.cs" />
     <Compile Include="Compute\Forked\ForkedResourceTaskTest.cs" />
     <Compile Include="Compute\Forked\ForkedSerializableClosureTaskTest.cs" />
@@ -137,6 +141,7 @@
     <Compile Include="ReconnectTest.cs" />
     <Compile Include="SerializationTest.cs" />
     <Compile Include="IgniteStartStopTest.cs" />
+    <Compile Include="Services\ServicesTestFullFooter.cs" />
     <Compile Include="TestUtils.cs" />
     <Compile Include="Memory\InteropMemoryTest.cs" />
     <Compile Include="Binary\BinaryBuilderSelfTest.cs" />
@@ -239,9 +244,6 @@
     <Content Include="Config\native-client-test-cache.xml">
       <CopyToOutputDirectory>Always</CopyToOutputDirectory>
     </Content>
-    <Content Include="Config\binary.xml">
-      <CopyToOutputDirectory>Always</CopyToOutputDirectory>
-    </Content>
     <Content Include="Config\start-test-grid1.xml">
       <CopyToOutputDirectory>Always</CopyToOutputDirectory>
     </Content>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryBuilderSelfTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryBuilderSelfTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryBuilderSelfTest.cs
index d442fb1..e2f7d8a 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryBuilderSelfTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryBuilderSelfTest.cs
@@ -24,6 +24,8 @@ namespace Apache.Ignite.Core.Tests.Binary
     using System.Collections.Generic;
     using System.Linq;
     using Apache.Ignite.Core.Binary;
+    using Apache.Ignite.Core.Discovery.Tcp;
+    using Apache.Ignite.Core.Discovery.Tcp.Static;
     using Apache.Ignite.Core.Impl;
     using Apache.Ignite.Core.Impl.Binary;
     using NUnit.Framework;
@@ -83,11 +85,18 @@ namespace Apache.Ignite.Core.Tests.Binary
                         new BinaryTypeConfiguration(TypeEmpty),
                         new BinaryTypeConfiguration(typeof(TestEnumRegistered))
                     },
-                    DefaultIdMapper = new IdMapper()
+                    DefaultIdMapper = new IdMapper(),
+                    CompactFooter = GetCompactFooter()
                 },
                 JvmClasspath = TestUtils.CreateTestClasspath(),
                 JvmOptions = TestUtils.TestJavaOptions(),
-                SpringConfigUrl = "config\\binary.xml"
+                DiscoverySpi = new TcpDiscoverySpi
+                {
+                    IpFinder = new TcpDiscoveryStaticIpFinder
+                    {
+                        Endpoints = new[] { "127.0.0.1:47500", "127.0.0.1:47501" }
+                    }
+                }
             };
 
             _grid = (Ignite) Ignition.Start(cfg);
@@ -96,10 +105,18 @@ namespace Apache.Ignite.Core.Tests.Binary
         }
 
         /// <summary>
+        /// Gets the compact footer setting.
+        /// </summary>
+        protected virtual bool GetCompactFooter()
+        {
+            return true;
+        }
+
+        /// <summary>
         /// Tear down routine.
         /// </summary>
         [TestFixtureTearDown]
-        public virtual void TearDown()
+        public void TearDown()
         {
             if (_grid != null)
                 Ignition.Stop(_grid.Name, true);
@@ -1418,6 +1435,16 @@ namespace Apache.Ignite.Core.Tests.Binary
                 Assert.AreEqual((TestEnumRegistered)val, binEnum.Deserialize<TestEnumRegistered>());
             }
         }
+
+        /// <summary>
+        /// Tests the compact footer setting.
+        /// </summary>
+        [Test]
+        public void TestCompactFooterSetting()
+        {
+            Assert.AreEqual(GetCompactFooter(), _marsh.CompactFooter);
+        }
+
     }
 
     /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryBuilderSelfTestFullFooter.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryBuilderSelfTestFullFooter.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryBuilderSelfTestFullFooter.cs
new file mode 100644
index 0000000..b6a1a8a
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryBuilderSelfTestFullFooter.cs
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.Core.Tests.Binary
+{
+    /// <summary>
+    /// Binary builder self test with compact footers disabled.
+    /// </summary>
+    public class BinaryBuilderSelfTestFullFooter : BinaryBuilderSelfTest
+    {
+        /** <inheritdoc /> */
+        protected override bool GetCompactFooter()
+        {
+            return false;
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryCompactFooterInteropTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryCompactFooterInteropTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryCompactFooterInteropTest.cs
new file mode 100644
index 0000000..b01b65e
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryCompactFooterInteropTest.cs
@@ -0,0 +1,129 @@
+/*
+ * 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.
+ */
+
+#pragma warning disable 618   // SpringConfigUrl
+namespace Apache.Ignite.Core.Tests.Binary
+{
+    using System.Collections;
+    using System.Linq;
+    using Apache.Ignite.Core.Binary;
+    using Apache.Ignite.Core.Tests.Compute;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests compact footer mode interop with Java.
+    /// </summary>
+    public class BinaryCompactFooterInteropTest
+    {
+        /** */
+        private const string PlatformSqlQueryTask = "org.apache.ignite.platform.PlatformSqlQueryTask";
+
+        /** */
+        private IIgnite _grid;
+
+        /** */
+        private IIgnite _clientGrid;
+
+        /// <summary>
+        /// Sets up the test.
+        /// </summary>
+        [SetUp]
+        public void TestSetUp()
+        {
+            // Start fresh cluster for each test
+            _grid = Ignition.Start(Config("config\\compute\\compute-grid1.xml"));
+            _clientGrid = Ignition.Start(Config("config\\compute\\compute-grid3.xml"));
+        }
+
+        /// <summary>
+        /// Tears down the test.
+        /// </summary>
+        [TearDown]
+        public void TestTearDown()
+        {
+            Ignition.StopAll(true);
+        }
+
+        /// <summary>
+        /// Tests an object that comes from Java.
+        /// </summary>
+        [Test]
+        public void TestFromJava([Values(true, false)] bool client)
+        {
+            var grid = client ? _clientGrid : _grid;
+
+            var fromJava = grid.GetCompute().ExecuteJavaTask<PlatformComputeBinarizable>(ComputeApiTest.EchoTask, 
+                ComputeApiTest.EchoTypeBinarizable);
+
+            Assert.AreEqual(1, fromJava.Field);
+        }
+
+        /// <summary>
+        /// Tests an object that comes from .NET in Java.
+        /// </summary>
+        [Test]
+        public void TestFromDotNet([Values(true, false)] bool client)
+        {
+            var grid = client ? _clientGrid : _grid;
+
+            var compute = grid.GetCompute().WithKeepBinary();
+
+            var arg = new PlatformComputeNetBinarizable {Field = 100};
+
+            var res = compute.ExecuteJavaTask<int>(ComputeApiTest.BinaryArgTask, arg);
+
+            Assert.AreEqual(arg.Field, res);
+        }
+
+        /// <summary>
+        /// Tests the indexing.
+        /// </summary>
+        [Test]
+        public void TestIndexing([Values(true, false)] bool client)
+        {
+            var grid = client ? _clientGrid : _grid;
+
+            var cache = grid.GetCache<int, PlatformComputeBinarizable>(null);
+
+            // Populate cache in .NET
+            for (var i = 0; i < 100; i++)
+                cache[i] = new PlatformComputeBinarizable {Field = i};
+
+            // Run SQL query on Java side
+            var qryRes = grid.GetCompute().ExecuteJavaTask<IList>(PlatformSqlQueryTask, "Field < 10");
+
+            Assert.AreEqual(10, qryRes.Count);
+            Assert.IsTrue(qryRes.OfType<PlatformComputeBinarizable>().All(x => x.Field < 10));
+        }
+
+        /// <summary>
+        /// Gets the config.
+        /// </summary>
+        private static IgniteConfiguration Config(string springUrl)
+        {
+            return new IgniteConfiguration
+            {
+                SpringConfigUrl = springUrl,
+                JvmOptions = TestUtils.TestJavaOptions(),
+                JvmClasspath = TestUtils.CreateTestClasspath(),
+                BinaryConfiguration = new BinaryConfiguration(
+                    typeof (PlatformComputeBinarizable),
+                    typeof (PlatformComputeNetBinarizable))
+            };
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinarySelfTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinarySelfTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinarySelfTest.cs
index 24ce3c8..0fcb792 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinarySelfTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinarySelfTest.cs
@@ -20,6 +20,7 @@
 // ReSharper disable PossibleInvalidOperationException
 // ReSharper disable UnusedAutoPropertyAccessor.Global
 // ReSharper disable MemberCanBePrivate.Global
+// ReSharper disable UnusedMember.Local
 namespace Apache.Ignite.Core.Tests.Binary
 {
     using System;
@@ -37,7 +38,7 @@ namespace Apache.Ignite.Core.Tests.Binary
     using BinaryWriter = Apache.Ignite.Core.Impl.Binary.BinaryWriter;
 
     /// <summary>
-    /// 
+    /// Binary tests.
     /// </summary>
     [TestFixture]
     public class BinarySelfTest { 
@@ -50,7 +51,16 @@ namespace Apache.Ignite.Core.Tests.Binary
         [TestFixtureSetUp]
         public void BeforeTest()
         {
-            _marsh = new Marshaller(null);
+            _marsh = new Marshaller(GetBinaryConfiguration());
+        }
+
+        /// <summary>
+        /// Gets the binary configuration.
+        /// </summary>
+        /// <returns></returns>
+        protected virtual BinaryConfiguration GetBinaryConfiguration()
+        {
+            return new BinaryConfiguration { CompactFooter = true };
         }
         
         /**
@@ -1418,6 +1428,15 @@ namespace Apache.Ignite.Core.Tests.Binary
             Assert.Throws<BinaryObjectException>(() => new Marshaller(cfg));
         }
 
+        /// <summary>
+        /// Tests the compact footer setting.
+        /// </summary>
+        [Test]
+        public void TestCompactFooterSetting()
+        {
+            Assert.AreEqual(GetBinaryConfiguration().CompactFooter, _marsh.CompactFooter);
+        }
+
         private static void CheckKeepSerialized(BinaryConfiguration cfg, bool expKeep)
         {
             if (cfg.TypeConfigurations == null)

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinarySelfTestFullFooter.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinarySelfTestFullFooter.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinarySelfTestFullFooter.cs
new file mode 100644
index 0000000..06e43e1
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinarySelfTestFullFooter.cs
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.Core.Tests.Binary
+{
+    using Apache.Ignite.Core.Binary;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Binary test with full footers.
+    /// </summary>
+    [TestFixture]
+    public class BinarySelfTestFullFooter : BinarySelfTest
+    {
+        /** <inheritdoc /> */
+        protected override BinaryConfiguration GetBinaryConfiguration()
+        {
+            return new BinaryConfiguration {CompactFooter = false};
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTest.cs
index c33d095..45fb4b4 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTest.cs
@@ -16,6 +16,8 @@
  */
 
 // ReSharper disable SpecifyACultureInStringConversionExplicitly
+// ReSharper disable UnusedAutoPropertyAccessor.Global
+#pragma warning disable 618  // SpringConfigUrl
 namespace Apache.Ignite.Core.Tests.Compute
 {
     using System;
@@ -27,6 +29,7 @@ namespace Apache.Ignite.Core.Tests.Compute
     using Apache.Ignite.Core.Cluster;
     using Apache.Ignite.Core.Common;
     using Apache.Ignite.Core.Compute;
+    using Apache.Ignite.Core.Impl;
     using Apache.Ignite.Core.Impl.Common;
     using Apache.Ignite.Core.Resource;
     using NUnit.Framework;
@@ -40,7 +43,7 @@ namespace Apache.Ignite.Core.Tests.Compute
         public const string EchoTask = "org.apache.ignite.platform.PlatformComputeEchoTask";
 
         /** Binary argument task name. */
-        private const string BinaryArgTask = "org.apache.ignite.platform.PlatformComputeBinarizableArgTask";
+        public const string BinaryArgTask = "org.apache.ignite.platform.PlatformComputeBinarizableArgTask";
 
         /** Broadcast task name. */
         public const string BroadcastTask = "org.apache.ignite.platform.PlatformComputeBroadcastTask";
@@ -88,7 +91,7 @@ namespace Apache.Ignite.Core.Tests.Compute
         private const int EchoTypeMap = 11;
 
         /** Echo type: binarizable. */
-        private const int EchoTypeBinarizable = 12;
+        public const int EchoTypeBinarizable = 12;
 
         /** Echo type: binary (Java only). */
         private const int EchoTypeBinarizableJava = 13;
@@ -126,14 +129,35 @@ namespace Apache.Ignite.Core.Tests.Compute
         [TestFixtureSetUp]
         public void InitClient()
         {
-            //TestUtils.JVM_DEBUG = true;
             TestUtils.KillProcesses();
 
-            _grid1 = Ignition.Start(Configuration("config\\compute\\compute-grid1.xml"));
-            _grid2 = Ignition.Start(Configuration("config\\compute\\compute-grid2.xml"));
-            _grid3 = Ignition.Start(Configuration("config\\compute\\compute-grid3.xml"));
+            var configs = GetConfigs();
+
+            _grid1 = Ignition.Start(Configuration(configs.Item1));
+            _grid2 = Ignition.Start(Configuration(configs.Item2));
+            _grid3 = Ignition.Start(Configuration(configs.Item3));
+        }
+
+        /// <summary>
+        /// Gets the configs.
+        /// </summary>
+        protected virtual Tuple<string, string, string> GetConfigs()
+        {
+            return Tuple.Create(
+                "config\\compute\\compute-grid1.xml",
+                "config\\compute\\compute-grid2.xml",
+                "config\\compute\\compute-grid3.xml");
+        }
+
+        /// <summary>
+        /// Gets the expected compact footers setting.
+        /// </summary>
+        protected virtual bool CompactFooter
+        {
+            get { return true; }
         }
 
+
         [TestFixtureTearDown]
         public void StopClient()
         {
@@ -313,7 +337,7 @@ namespace Apache.Ignite.Core.Tests.Compute
 
             Assert.AreEqual(topVer + 1, _grid1.GetCluster().TopologyVersion);
 
-            _grid3 = Ignition.Start(Configuration("config\\compute\\compute-grid3.xml"));
+            _grid3 = Ignition.Start(Configuration(GetConfigs().Item3));
 
             Assert.AreEqual(topVer + 2, _grid1.GetCluster().TopologyVersion);
         }
@@ -354,7 +378,7 @@ namespace Apache.Ignite.Core.Tests.Compute
             }
             finally 
             {
-                _grid2 = Ignition.Start(Configuration("config\\compute\\compute-grid2.xml"));
+                _grid2 = Ignition.Start(Configuration(GetConfigs().Item2));
             }
         }
 
@@ -386,7 +410,7 @@ namespace Apache.Ignite.Core.Tests.Compute
 
             Assert.IsTrue(nodes.Count == 2);
 
-            _grid2 = Ignition.Start(Configuration("config\\compute\\compute-grid2.xml"));
+            _grid2 = Ignition.Start(Configuration(GetConfigs().Item2));
 
             nodes = _grid1.GetCluster().GetNodes();
 
@@ -1135,6 +1159,9 @@ namespace Apache.Ignite.Core.Tests.Compute
             Assert.AreEqual(_grid1.GetCompute().ClusterGroup.GetNodes().Count, res);
         }
 
+        /// <summary>
+        /// Tests the exceptions.
+        /// </summary>
         [Test]
         public void TestExceptions()
         {
@@ -1146,6 +1173,18 @@ namespace Apache.Ignite.Core.Tests.Compute
         }
 
         /// <summary>
+        /// Tests the footer setting.
+        /// </summary>
+        [Test]
+        public void TestFooterSetting()
+        {
+            Assert.AreEqual(CompactFooter, ((Ignite)_grid1).Marshaller.CompactFooter);
+
+            foreach (var g in new[] {_grid1, _grid2, _grid3})
+                Assert.AreEqual(CompactFooter, g.GetConfiguration().BinaryConfiguration.CompactFooter);
+        }
+
+        /// <summary>
         /// Create configuration.
         /// </summary>
         /// <param name="path">XML config path.</param>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTestFullFooter.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTestFullFooter.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTestFullFooter.cs
new file mode 100644
index 0000000..1b1f98a
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTestFullFooter.cs
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.Core.Tests.Compute
+{
+    using System;
+    using System.IO;
+
+    /// <summary>
+    /// Compute API test with compact footers disabled.
+    /// </summary>
+    public class ComputeApiTestFullFooter : ComputeApiTest
+    {
+        /// <summary>
+        /// Gets the expected compact footers setting.
+        /// </summary>
+        protected override bool CompactFooter
+        {
+            get { return false; }
+        }
+
+        /// <summary>
+        /// Gets the configs.
+        /// </summary>
+        protected override Tuple<string, string, string> GetConfigs()
+        {
+            var baseConfigs = base.GetConfigs();
+
+            return Tuple.Create(
+                ReplaceFooterSetting(baseConfigs.Item1),
+                ReplaceFooterSetting(baseConfigs.Item2),
+                ReplaceFooterSetting(baseConfigs.Item3));
+        }
+
+        /// <summary>
+        /// Replaces the footer setting.
+        /// </summary>
+        public static string ReplaceFooterSetting(string path)
+        {
+            var text = File.ReadAllText(path).Replace(
+                "property name=\"compactFooter\" value=\"true\"",
+                "property name=\"compactFooter\" value=\"false\"");
+
+            path += "_fullFooter";
+
+            File.WriteAllText(path, text);
+
+            return path;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/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 78a30a8..566d6ac 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
@@ -19,8 +19,11 @@
 
 <beans xmlns="http://www.springframework.org/schema/beans"
        xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xmlns:util="http://www.springframework.org/schema/util"
        xsi:schemaLocation="http://www.springframework.org/schema/beans
-        http://www.springframework.org/schema/beans/spring-beans.xsd">
+                           http://www.springframework.org/schema/beans/spring-beans.xsd
+                           http://www.springframework.org/schema/util
+                           http://www.springframework.org/schema/util/spring-util.xsd">
     <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
         <property name="localHost" value="127.0.0.1"/>
         <property name="connectorConfiguration"><null/></property>
@@ -40,6 +43,19 @@
             <list>
                 <bean class="org.apache.ignite.configuration.CacheConfiguration">
                     <property name="startSize" value="10"/>
+                    <property name="queryEntities">
+                        <list>
+                            <bean class="org.apache.ignite.cache.QueryEntity">
+                                <property name="keyType" value="java.lang.Integer"/>
+                                <property name="valueType" value="org.apache.ignite.platform.PlatformComputeBinarizable"/>
+                                <property name="fields">
+                                    <util:map map-class="java.util.LinkedHashMap">
+                                        <entry key="Field" value="java.lang.Integer"/>
+                                    </util:map>
+                                </property>
+                            </bean>
+                        </list>
+                    </property>
                 </bean>
                 <bean class="org.apache.ignite.configuration.CacheConfiguration">
                     <property name="name" value="cache1"/>
@@ -54,7 +70,7 @@
 
         <property name="binaryConfiguration">
             <bean class="org.apache.ignite.configuration.BinaryConfiguration">
-                <property name="compactFooter" value="false"/>
+                <property name="compactFooter" value="true"/>
                 <property name="typeConfigurations">
                     <list>
                         <bean class="org.apache.ignite.binary.BinaryTypeConfiguration">

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid2.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid2.xml b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid2.xml
index b1e8235..711c3e3 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid2.xml
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid2.xml
@@ -47,7 +47,7 @@
 
         <property name="binaryConfiguration">
             <bean class="org.apache.ignite.configuration.BinaryConfiguration">
-                <property name="compactFooter" value="false"/>
+                <property name="compactFooter" value="true"/>
                 <property name="typeConfigurations">
                     <list>
                         <bean class="org.apache.ignite.binary.BinaryTypeConfiguration">

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid3.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid3.xml b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid3.xml
index d1c96b6..64d14bb 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid3.xml
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid3.xml
@@ -34,7 +34,7 @@
 
         <property name="binaryConfiguration">
             <bean class="org.apache.ignite.configuration.BinaryConfiguration">
-                <property name="compactFooter" value="false"/>
+                <property name="compactFooter" value="true"/>
                 <property name="typeConfigurations">
                     <list>
                         <bean class="org.apache.ignite.binary.BinaryTypeConfiguration">

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/binary.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/binary.xml b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/binary.xml
deleted file mode 100644
index f013749..0000000
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/binary.xml
+++ /dev/null
@@ -1,56 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-
-<!--
-  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.
--->
-
-<beans xmlns="http://www.springframework.org/schema/beans"
-       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-       xsi:schemaLocation="http://www.springframework.org/schema/beans
-        http://www.springframework.org/schema/beans/spring-beans.xsd">
-    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
-        <property name="localHost" value="127.0.0.1"/>
-        <property name="connectorConfiguration"><null/></property>
-
-        <property name="gridName" value="grid"/>
-
-        <property name="metricsUpdateFrequency" value="1000"/>
-        <property name="metricsLogFrequency" value="0"/>
-
-        <property name="cacheConfiguration">
-            <list>
-                <bean class="org.apache.ignite.configuration.CacheConfiguration">
-                    <property name="name" value="cache"/>
-                </bean>
-            </list>
-        </property>
-      
-        <property name="discoverySpi">
-            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
-                <property name="ipFinder">
-                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
-                        <property name="addresses">
-                            <list>
-                                <!-- In distributed environment, replace with actual host IP address. -->
-                                <value>127.0.0.1:47500..47502</value>
-                            </list>
-                        </property>
-                    </bean>
-                </property>
-            </bean>
-        </property>
-    </bean>
-</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
index d8c52ee..d944a04 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
@@ -54,7 +54,7 @@ namespace Apache.Ignite.Core.Tests
         {
             var xml = @"<igniteConfig workDirectory='c:' JvmMaxMemoryMb='1024' MetricsLogFrequency='0:0:10'>
                             <localhost>127.1.1.1</localhost>
-                            <binaryConfiguration>
+                            <binaryConfiguration compactFooter='false'>
                                 <defaultNameMapper type='Apache.Ignite.Core.Tests.IgniteConfigurationSerializerTest+NameMapper, Apache.Ignite.Core.Tests' bar='testBar' />
                                 <types>
                                     <string>Apache.Ignite.Core.Tests.IgniteConfigurationSerializerTest+FooClass, Apache.Ignite.Core.Tests</string>
@@ -112,6 +112,7 @@ namespace Apache.Ignite.Core.Tests
             Assert.AreEqual(
                 "Apache.Ignite.Core.Tests.IgniteConfigurationSerializerTest+FooClass, Apache.Ignite.Core.Tests",
                 cfg.BinaryConfiguration.Types.Single());
+            Assert.IsFalse(cfg.BinaryConfiguration.CompactFooter);
             Assert.AreEqual(new[] {42, EventType.TaskFailed, EventType.JobFinished}, cfg.IncludedEventTypes);
 
             Assert.AreEqual("secondCache", cfg.CacheConfiguration.Last().Name);

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
index 3aa26d8..c1f8fcd 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
@@ -22,6 +22,7 @@ namespace Apache.Ignite.Core.Tests
     using System.ComponentModel;
     using System.IO;
     using System.Linq;
+    using Apache.Ignite.Core.Binary;
     using Apache.Ignite.Core.Cache.Configuration;
     using Apache.Ignite.Core.Common;
     using Apache.Ignite.Core.Discovery.Tcp;
@@ -60,6 +61,7 @@ namespace Apache.Ignite.Core.Tests
         public void TestDefaultValueAttributes()
         {
             CheckDefaultValueAttributes(new IgniteConfiguration());
+            CheckDefaultValueAttributes(new BinaryConfiguration());
             CheckDefaultValueAttributes(new TcpDiscoverySpi());
             CheckDefaultValueAttributes(new CacheConfiguration());
             CheckDefaultValueAttributes(new TcpDiscoveryMulticastIpFinder());

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTest.cs
index ffcdea8..c563a61 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTest.cs
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+#pragma warning disable 618   // SpringConfigUrl
 namespace Apache.Ignite.Core.Tests.Services
 {
     using System;
@@ -24,8 +25,10 @@ namespace Apache.Ignite.Core.Tests.Services
     using Apache.Ignite.Core.Binary;
     using Apache.Ignite.Core.Cluster;
     using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Impl;
     using Apache.Ignite.Core.Resource;
     using Apache.Ignite.Core.Services;
+    using Apache.Ignite.Core.Tests.Compute;
     using NUnit.Framework;
 
     /// <summary>
@@ -507,6 +510,19 @@ namespace Apache.Ignite.Core.Tests.Services
         }
 
         /// <summary>
+        /// Tests the footer setting.
+        /// </summary>
+        [Test]
+        public void TestFooterSetting()
+        {
+            foreach (var grid in Grids)
+            {
+                Assert.AreEqual(CompactFooter, ((Ignite)grid).Marshaller.CompactFooter);
+                Assert.AreEqual(CompactFooter, grid.GetConfiguration().BinaryConfiguration.CompactFooter);
+            }
+        }
+
+        /// <summary>
         /// Starts the grids.
         /// </summary>
         private void StartGrids()
@@ -558,8 +574,11 @@ namespace Apache.Ignite.Core.Tests.Services
         /// <summary>
         /// Gets the Ignite configuration.
         /// </summary>
-        private static IgniteConfiguration Configuration(string springConfigUrl)
+        private IgniteConfiguration Configuration(string springConfigUrl)
         {
+            if (!CompactFooter)
+                springConfigUrl = ComputeApiTestFullFooter.ReplaceFooterSetting(springConfigUrl);
+
             return new IgniteConfiguration
             {
                 SpringConfigUrl = springConfigUrl,
@@ -598,6 +617,11 @@ namespace Apache.Ignite.Core.Tests.Services
         }
 
         /// <summary>
+        /// Gets a value indicating whether compact footers should be used.
+        /// </summary>
+        protected virtual bool CompactFooter { get { return true; } }
+
+        /// <summary>
         /// Test service interface for proxying.
         /// </summary>
         private interface ITestIgniteService

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTestFullFooter.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTestFullFooter.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTestFullFooter.cs
new file mode 100644
index 0000000..b4b4f11
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTestFullFooter.cs
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.Core.Tests.Services
+{
+    /// <summary>
+    /// Services test with compact footers disabled.
+    /// </summary>
+    public class ServicesTestFullFooter : ServicesTest
+    {
+        /// <summary>
+        /// Gets a value indicating whether compact footers should be used.
+        /// </summary>
+        protected override bool CompactFooter
+        {
+            get { return false; }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
index f1511d9..dedf084 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
@@ -193,6 +193,7 @@
     <Compile Include="Ignition.cs" />
     <Compile Include="IIgnite.cs" />
     <Compile Include="Impl\Binary\BinaryEnum.cs" />
+    <Compile Include="Impl\Binary\BinaryObjectSchemaSerializer.cs" />
     <Compile Include="Impl\Binary\JavaTypes.cs" />
     <Compile Include="Impl\Cache\CacheAffinityImpl.cs" />
     <Compile Include="Impl\Cache\CacheEntry.cs" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/platforms/dotnet/Apache.Ignite.Core/Binary/BinaryConfiguration.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Binary/BinaryConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Binary/BinaryConfiguration.cs
index fa2fb1c..3a9d86c 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Binary/BinaryConfiguration.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Binary/BinaryConfiguration.cs
@@ -19,6 +19,7 @@ namespace Apache.Ignite.Core.Binary
 {
     using System;
     using System.Collections.Generic;
+    using System.ComponentModel;
     using System.Diagnostics.CodeAnalysis;
     using System.Linq;
     using Apache.Ignite.Core.Impl.Common;
@@ -29,11 +30,24 @@ namespace Apache.Ignite.Core.Binary
     public class BinaryConfiguration
     {
         /// <summary>
+        /// Default <see cref="CompactFooter"/> setting.
+        /// </summary>
+        public const bool DefaultCompactFooter = true;
+
+        /// <summary>
+        /// Default <see cref="DefaultKeepDeserialized"/> setting.
+        /// </summary>
+        public const bool DefaultDefaultKeepDeserialized = true;
+
+        /** Footer setting. */
+        private bool? _compactFooter;
+
+        /// <summary>
         /// Initializes a new instance of the <see cref="BinaryConfiguration"/> class.
         /// </summary>
         public BinaryConfiguration()
         {
-            DefaultKeepDeserialized = true;
+            DefaultKeepDeserialized = DefaultDefaultKeepDeserialized;
         }
 
         /// <summary>
@@ -54,6 +68,8 @@ namespace Apache.Ignite.Core.Binary
                 : cfg.TypeConfigurations.Select(x => new BinaryTypeConfiguration(x)).ToList();
 
             Types = cfg.Types == null ? null : cfg.Types.ToList();
+
+            CompactFooter = cfg.CompactFooter;
         }
 
         /// <summary>
@@ -95,6 +111,32 @@ namespace Apache.Ignite.Core.Binary
         /// <summary>
         /// Default keep deserialized flag.
         /// </summary>
+        [DefaultValue(DefaultDefaultKeepDeserialized)]
         public bool DefaultKeepDeserialized { get; set; }
+
+        /// <summary>
+        /// Gets or sets a value indicating whether to write footers in compact form.
+        /// When enabled, Ignite will not write fields metadata when serializing objects, 
+        /// because internally metadata is distributed inside cluster.
+        /// This increases serialization performance.
+        /// <para/>
+        /// <b>WARNING!</b> This mode should be disabled when already serialized data can be taken from some external
+        /// sources (e.g.cache store which stores data in binary form, data center replication, etc.). 
+        /// Otherwise binary objects without any associated metadata could could not be deserialized.
+        /// </summary>
+        [DefaultValue(DefaultCompactFooter)]
+        public bool CompactFooter
+        {
+            get { return _compactFooter ?? DefaultCompactFooter; }
+            set { _compactFooter = value; }
+        }
+
+        /// <summary>
+        /// Gets the compact footer internal nullable value.
+        /// </summary>
+        internal bool? CompactFooterInternal
+        {
+            get { return _compactFooter; }
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs
index 1dd22ea..e8cc8ff 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs
@@ -207,6 +207,14 @@
             }
             else
                 writer.WriteBoolean(false);
+
+            // Binary config
+            var isCompactFooterSet = BinaryConfiguration != null && BinaryConfiguration.CompactFooterInternal != null;
+
+            writer.WriteBoolean(isCompactFooterSet);
+
+            if (isCompactFooterSet)
+                writer.WriteBoolean(BinaryConfiguration.CompactFooter);
         }
 
         /// <summary>
@@ -237,6 +245,13 @@
 
             // Discovery config
             DiscoverySpi = r.ReadBoolean() ? new TcpDiscoverySpi(r) : null;
+
+            // Binary config
+            if (r.ReadBoolean())
+            {
+                BinaryConfiguration = BinaryConfiguration ?? new BinaryConfiguration();
+                BinaryConfiguration.CompactFooter = r.ReadBoolean();
+            }
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
index 5181217..12a4660 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
@@ -78,6 +78,7 @@
                             </xs:element>
                         </xs:all>
                         <xs:attribute name="defaultKeepDeserialized" type="xs:boolean" />
+                        <xs:attribute name="compactFooter" type="xs:boolean" />
                     </xs:complexType>
                 </xs:element>
                 <xs:element name="cacheConfiguration" minOccurs="0">

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Binary.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Binary.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Binary.cs
index efe1df4..1deac07 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Binary.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Binary.cs
@@ -190,8 +190,12 @@ namespace Apache.Ignite.Core.Impl.Binary
         {
             const int len = BinaryObjectHeader.Size;
 
-            var hdr = new BinaryObjectHeader(desc.TypeId, 0, len, 0, len,
-                desc.UserType ? BinaryObjectHeader.Flag.UserType : BinaryObjectHeader.Flag.None);
+            var flags = desc.UserType ? BinaryObjectHeader.Flag.UserType : BinaryObjectHeader.Flag.None;
+
+            if (_marsh.CompactFooter && desc.UserType)
+                flags |= BinaryObjectHeader.Flag.CompactFooter;
+
+            var hdr = new BinaryObjectHeader(desc.TypeId, 0, len, 0, len, flags);
 
             using (var stream = new BinaryHeapStream(len))
             {

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObject.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObject.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObject.cs
index 513333b..13d3133 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObject.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObject.cs
@@ -197,7 +197,7 @@ namespace Apache.Ignite.Core.Impl.Binary
         {
             var desc = _marsh.GetDescriptor(true, _header.TypeId);
 
-            InitializeFields();
+            InitializeFields(desc);
 
             int fieldId = BinaryUtils.FieldId(_header.TypeId, fieldName, desc.NameMapper, desc.IdMapper);
 
@@ -207,16 +207,19 @@ namespace Apache.Ignite.Core.Impl.Binary
         /// <summary>
         /// Lazy fields initialization routine.
         /// </summary>
-        private void InitializeFields()
+        private void InitializeFields(IBinaryTypeDescriptor desc = null)
         {
             if (_fields != null) 
                 return;
 
+            desc = desc ?? _marsh.GetDescriptor(true, _header.TypeId);
+
             using (var stream = new BinaryHeapStream(_data))
             {
                 var hdr = BinaryObjectHeader.Read(stream, _offset);
 
-                _fields = hdr.ReadSchemaAsDictionary(stream, _offset) ?? EmptyFields;
+                _fields = BinaryObjectSchemaSerializer.ReadSchema(stream, _offset, hdr, desc.Schema,_marsh)
+                    .ToDictionary() ?? EmptyFields;
             }
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/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 0f1c0bd..083f557 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectBuilder.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectBuilder.cs
@@ -632,7 +632,8 @@ namespace Apache.Ignite.Core.Impl.Binary
                     else
                     {
                         // New object, write in full form.
-                        var inSchema = inHeader.ReadSchema(inStream, inStartPos);
+                        var inSchema = BinaryObjectSchemaSerializer.ReadSchema(inStream, inStartPos, inHeader, 
+                            _desc.Schema, _binary.Marshaller);
 
                         var outSchema = BinaryObjectSchemaHolder.Current;
                         var schemaIdx = outSchema.PushSchema();
@@ -709,6 +710,9 @@ namespace Apache.Ignite.Core.Impl.Binary
                             var schemaPos = outStream.Position;
                             int outSchemaId;
 
+                            if (inHeader.IsCompactFooter)
+                                flags |= BinaryObjectHeader.Flag.CompactFooter;
+
                             var hasSchema = outSchema.WriteSchema(outStream, schemaIdx, out outSchemaId, ref flags);
 
                             if (hasSchema)
@@ -719,6 +723,9 @@ namespace Apache.Ignite.Core.Impl.Binary
 
                                 if (inHeader.HasRaw)
                                     outStream.WriteInt(outRawOff);
+
+                                if (_desc.Schema.Get(outSchemaId) == null)
+                                    _desc.Schema.Add(outSchemaId, outSchema.GetSchema(schemaIdx));
                             }
 
                             var outLen = outStream.Position - outStartPos;

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/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 0cabd7d..2624d52 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectHeader.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectHeader.cs
@@ -18,7 +18,6 @@
 namespace Apache.Ignite.Core.Impl.Binary
 {
     using System;
-    using System.Collections.Generic;
     using System.Diagnostics;
     using System.IO;
     using System.Runtime.InteropServices;
@@ -183,7 +182,7 @@ namespace Apache.Ignite.Core.Impl.Binary
         /// </summary>
         public int SchemaFieldSize
         {
-            get { return SchemaFieldOffsetSize + 4; }
+            get { return IsCompactFooter ? SchemaFieldOffsetSize : SchemaFieldOffsetSize + 4; }
         }
 
         /// <summary>
@@ -198,6 +197,9 @@ namespace Apache.Ignite.Core.Impl.Binary
 
                 var schemaSize = Length - SchemaOffset;
 
+                if (HasRaw)
+                    schemaSize -= 4;
+
                 return schemaSize / SchemaFieldSize;
             }
         }
@@ -221,164 +223,6 @@ namespace Apache.Ignite.Core.Impl.Binary
         }
 
         /// <summary>
-        /// Reads the schema as dictionary according to this header data.
-        /// </summary>
-        /// <param name="stream">The stream.</param>
-        /// <param name="position">The position.</param>
-        /// <returns>Schema.</returns>
-        public Dictionary<int, int> ReadSchemaAsDictionary(IBinaryStream stream, int position)
-        {
-            Debug.Assert(stream != null);
-
-            ThrowIfUnsupported();
-
-            var schemaSize = SchemaFieldCount;
-
-            if (schemaSize == 0)
-                return null;
-
-            stream.Seek(position + SchemaOffset, SeekOrigin.Begin);
-
-            var schema = new Dictionary<int, int>(schemaSize);
-
-            var offsetSize = SchemaFieldOffsetSize;
-
-            if (offsetSize == 1)
-            {
-                for (var i = 0; i < schemaSize; i++)
-                    schema.Add(stream.ReadInt(), stream.ReadByte());
-            }
-            else if (offsetSize == 2)
-            {
-                for (var i = 0; i < schemaSize; i++)
-                    schema.Add(stream.ReadInt(), stream.ReadShort());
-            }
-            else
-            {
-                for (var i = 0; i < schemaSize; i++)
-                    schema.Add(stream.ReadInt(), stream.ReadInt());
-            }
-
-            return schema;
-        }
-
-        /// <summary>
-        /// Reads the schema according to this header data.
-        /// </summary>
-        /// <param name="stream">The stream.</param>
-        /// <param name="position">The position.</param>
-        /// <returns>Schema.</returns>
-        public BinaryObjectSchemaField[] ReadSchema(IBinaryStream stream, int position)
-        {
-            Debug.Assert(stream != null);
-
-            ThrowIfUnsupported();
-
-            var schemaSize = SchemaFieldCount;
-
-            if (schemaSize == 0)
-                return null;
-
-            stream.Seek(position + SchemaOffset, SeekOrigin.Begin);
-
-            var schema = new BinaryObjectSchemaField[schemaSize];
-
-            var offsetSize = SchemaFieldOffsetSize;
-
-            if (offsetSize == 1)
-            {
-                for (var i = 0; i < schemaSize; i++)
-                    schema[i] = new BinaryObjectSchemaField(stream.ReadInt(), stream.ReadByte());
-            }
-            else if (offsetSize == 2)
-            {
-                for (var i = 0; i < schemaSize; i++)
-                    schema[i] = new BinaryObjectSchemaField(stream.ReadInt(), stream.ReadShort());
-            }
-            else
-            {
-                for (var i = 0; i < schemaSize; i++)
-                    schema[i] = new BinaryObjectSchemaField(stream.ReadInt(), stream.ReadInt());
-            }
-
-            return schema;
-        }
-
-        /// <summary>
-        /// Writes an array of fields to a stream.
-        /// </summary>
-        /// <param name="fields">Fields.</param>
-        /// <param name="stream">Stream.</param>
-        /// <param name="offset">Offset in the array.</param>
-        /// <param name="count">Field count to write.</param>
-        /// <returns>
-        /// Flags according to offset sizes: <see cref="Flag.OffsetOneByte" />,
-        /// <see cref="Flag.OffsetTwoBytes" />, or 0.
-        /// </returns>
-        public static unsafe Flag WriteSchema(BinaryObjectSchemaField[] fields, IBinaryStream stream, int offset,
-            int count)
-        {
-            Debug.Assert(fields != null);
-            Debug.Assert(stream != null);
-            Debug.Assert(count > 0);
-            Debug.Assert(offset >= 0);
-            Debug.Assert(offset < fields.Length);
-
-            unchecked
-            {
-                // Last field is the farthest in the stream
-                var maxFieldOffset = fields[offset + count - 1].Offset;
-
-                if (maxFieldOffset <= byte.MaxValue)
-                {
-                    for (int i = offset; i < count + offset; i++)
-                    {
-                        var field = fields[i];
-
-                        stream.WriteInt(field.Id);
-                        stream.WriteByte((byte)field.Offset);
-                    }
-
-                    return Flag.OffsetOneByte;
-                }
-
-                if (maxFieldOffset <= ushort.MaxValue)
-                {
-                    for (int i = offset; i < count + offset; i++)
-                    {
-                        var field = fields[i];
-
-                        stream.WriteInt(field.Id);
-
-                        stream.WriteShort((short)field.Offset);
-                    }
-
-                    return Flag.OffsetTwoBytes;
-                }
-
-                if (BitConverter.IsLittleEndian)
-                {
-                    fixed (BinaryObjectSchemaField* ptr = &fields[offset])
-                    {
-                        stream.Write((byte*)ptr, count / BinaryObjectSchemaField.Size);
-                    }
-                }
-                else
-                {
-                    for (int i = offset; i < count + offset; i++)
-                    {
-                        var field = fields[i];
-
-                        stream.WriteInt(field.Id);
-                        stream.WriteInt(field.Offset);
-                    }
-                }
-
-                return Flag.None;
-            }
-        }
-
-        /// <summary>
         /// Writes specified header to a stream.
         /// </summary>
         /// <param name="header">The header.</param>
@@ -424,8 +268,6 @@ namespace Apache.Ignite.Core.Impl.Binary
             else
                 hdr = new BinaryObjectHeader(stream);
 
-            hdr.ThrowIfUnsupported();
-
             // Only one of the flags can be set
             var f = hdr.Flags;
             Debug.Assert((f & (Flag.OffsetOneByte | Flag.OffsetTwoBytes)) !=
@@ -482,15 +324,5 @@ 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/c6c93892/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchema.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchema.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchema.cs
index a3467b8..9b12fa7 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchema.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchema.cs
@@ -94,5 +94,25 @@ namespace Apache.Ignite.Core.Impl.Binary
                 }
             }
         }
+
+        /// <summary>
+        /// Gets all schemas.
+        /// </summary>
+        public IEnumerable<KeyValuePair<int, int[]>> GetAll()
+        {
+            if (_schema1 == null)
+                yield break;
+
+            yield return new KeyValuePair<int, int[]>(_schemaId1, _schema1);
+
+            if (_schema2 == null)
+                yield break;
+
+            yield return new KeyValuePair<int, int[]>(_schemaId2, _schema2);
+
+            if (_schemas != null)
+                foreach (var pair in _schemas)
+                    yield return pair;
+        }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/c6c93892/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 65b6fc0..c95746a 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchemaHolder.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchemaHolder.cs
@@ -96,12 +96,28 @@ namespace Apache.Ignite.Core.Impl.Binary
             if (count == 0) 
                 return false;
 
-            flags |= BinaryObjectHeader.WriteSchema(_fields, stream, schemaOffset, count);
+            flags |= BinaryObjectSchemaSerializer.WriteSchema(_fields, stream, schemaOffset, count, 
+                (flags & BinaryObjectHeader.Flag.CompactFooter) == BinaryObjectHeader.Flag.CompactFooter);
 
             for (var i = schemaOffset; i < _idx; i++)
                 schemaId = Fnv1Hash.Update(schemaId, _fields[i].Id);
 
             return true;
         }
+
+        /// <summary>
+        /// Gets the schema.
+        /// </summary>
+        /// <param name="schemaOffset">The schema offset.</param>
+        /// <returns>Current schema as an array of field ids.</returns>
+        public int[] GetSchema(int schemaOffset)
+        {
+            int[] result = new int[_idx - schemaOffset];
+
+            for (int i = 0; i < result.Length; i++)
+                result[i] = _fields[i + schemaOffset].Id;
+
+            return result;
+        }
     }
 }