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

[11/50] [abbrv] ignite git commit: IGNITE-1956: Added binary enums support.

http://git-wip-us.apache.org/repos/asf/ignite/blob/663e78dc/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 ea472eb..373e173 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryBuilderSelfTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryBuilderSelfTest.cs
@@ -80,24 +80,13 @@ namespace Apache.Ignite.Core.Tests.Binary
                         new BinaryTypeConfiguration(typeof (BuilderCollection)),
                         new BinaryTypeConfiguration(typeof (BuilderCollectionItem)),
                         new BinaryTypeConfiguration(typeof (DecimalHolder)),
-                        new BinaryTypeConfiguration(TypeEmpty)
+                        new BinaryTypeConfiguration(TypeEmpty),
+                        new BinaryTypeConfiguration(typeof(TestEnumRegistered))
                     },
                     DefaultIdMapper = new IdMapper()
                 },
                 JvmClasspath = TestUtils.CreateTestClasspath(),
-                JvmOptions = new List<string>
-                {
-                    "-ea",
-                    "-Xcheck:jni",
-                    "-Xms4g",
-                    "-Xmx4g",
-                    "-DIGNITE_QUIET=false",
-                    "-Xnoagent",
-                    "-Djava.compiler=NONE",
-                    "-Xdebug",
-                    "-Xrunjdwp:transport=dt_socket,server=y,suspend=n,address=5005",
-                    "-XX:+HeapDumpOnOutOfMemoryError"
-                },
+                JvmOptions = TestUtils.TestJavaOptions(),
                 SpringConfigUrl = "config\\binary.xml"
             };
 
@@ -198,7 +187,7 @@ namespace Apache.Ignite.Core.Tests.Binary
             DateTime date = DateTime.Now.ToUniversalTime();
             Guid guid = Guid.NewGuid();
 
-            IIgniteBinary api = _grid.GetBinary();
+            IBinary api = _grid.GetBinary();
 
             // 1. Primitives.
             Assert.AreEqual(1, api.ToBinary<byte>((byte)1));
@@ -216,7 +205,8 @@ namespace Apache.Ignite.Core.Tests.Binary
             Assert.AreEqual("a", api.ToBinary<string>("a"));
             Assert.AreEqual(date, api.ToBinary<DateTime>(date));
             Assert.AreEqual(guid, api.ToBinary<Guid>(guid));
-            Assert.AreEqual(TestEnum.One, api.ToBinary<TestEnum>(TestEnum.One));
+            Assert.AreEqual(TestEnumRegistered.One, api.ToBinary<IBinaryObject>(TestEnumRegistered.One)
+                .Deserialize<TestEnumRegistered>());
 
             // 3. Arrays.
             Assert.AreEqual(new byte[] { 1 }, api.ToBinary<byte[]>(new byte[] { 1 }));
@@ -233,7 +223,9 @@ namespace Apache.Ignite.Core.Tests.Binary
             Assert.AreEqual(new[] { "a" }, api.ToBinary<string[]>(new[] { "a" }));
             Assert.AreEqual(new[] { date }, api.ToBinary<DateTime[]>(new[] { date }));
             Assert.AreEqual(new[] { guid }, api.ToBinary<Guid[]>(new[] { guid }));
-            Assert.AreEqual(new[] { TestEnum.One }, api.ToBinary<TestEnum[]>(new[] { TestEnum.One }));
+            Assert.AreEqual(new[] { TestEnumRegistered.One},
+                api.ToBinary<IBinaryObject[]>(new[] { TestEnumRegistered.One})
+                .Select(x => x.Deserialize<TestEnumRegistered>()).ToArray());
 
             // 4. Objects.
             IBinaryObject binObj = api.ToBinary<IBinaryObject>(new ToBinary(1));
@@ -816,7 +808,7 @@ namespace Apache.Ignite.Core.Tests.Binary
             Assert.AreEqual(new[] { "str" }, binObj.GetField<string[]>("fStrArr"));
             Assert.AreEqual(new[] { nDate }, binObj.GetField<DateTime?[]>("fDateArr"));
             Assert.AreEqual(new[] { nGuid }, binObj.GetField<Guid?[]>("fGuidArr"));
-            Assert.AreEqual(new[] { TestEnum.One }, binObj.GetField<TestEnum[]>("fEnumArr"));
+            Assert.AreEqual(new[] {TestEnum.One}, binObj.GetField<TestEnum[]>("fEnumArr"));
 
             StringDateGuidEnum obj = binObj.Deserialize<StringDateGuidEnum>();
 
@@ -839,7 +831,7 @@ namespace Apache.Ignite.Core.Tests.Binary
             Assert.AreEqual(new[] { "str" }, builder.GetField<string[]>("fStrArr"));
             Assert.AreEqual(new[] { nDate }, builder.GetField<DateTime?[]>("fDateArr"));
             Assert.AreEqual(new[] { nGuid }, builder.GetField<Guid?[]>("fGuidArr"));
-            Assert.AreEqual(new[] { TestEnum.One }, builder.GetField<TestEnum[]>("fEnumArr"));
+            Assert.AreEqual(new[] {TestEnum.One}, builder.GetField<TestEnum[]>("fEnumArr"));
 
             // Check reassemble.
             binObj = builder.Build();
@@ -851,7 +843,7 @@ namespace Apache.Ignite.Core.Tests.Binary
             Assert.AreEqual(new[] { "str" }, binObj.GetField<string[]>("fStrArr"));
             Assert.AreEqual(new[] { nDate }, binObj.GetField<DateTime?[]>("fDateArr"));
             Assert.AreEqual(new[] { nGuid }, binObj.GetField<Guid?[]>("fGuidArr"));
-            Assert.AreEqual(new[] { TestEnum.One }, binObj.GetField<TestEnum[]>("fEnumArr"));
+            Assert.AreEqual(new[] {TestEnum.One}, binObj.GetField<TestEnum[]>("fEnumArr"));
 
             obj = binObj.Deserialize<StringDateGuidEnum>();
 
@@ -889,7 +881,7 @@ namespace Apache.Ignite.Core.Tests.Binary
             Assert.AreEqual(new[] { "str2" }, binObj.GetField<string[]>("fStrArr"));
             Assert.AreEqual(new[] { nDate }, binObj.GetField<DateTime?[]>("fDateArr"));
             Assert.AreEqual(new[] { nGuid }, binObj.GetField<Guid?[]>("fGuidArr"));
-            Assert.AreEqual(new[] { TestEnum.Two }, binObj.GetField<TestEnum[]>("fEnumArr"));
+            Assert.AreEqual(new[] {TestEnum.Two}, binObj.GetField<TestEnum[]>("fEnumArr"));
 
             obj = binObj.Deserialize<StringDateGuidEnum>();
 
@@ -903,6 +895,24 @@ namespace Apache.Ignite.Core.Tests.Binary
             Assert.AreEqual(new[] { TestEnum.Two }, obj.FEnumArr);
         }
 
+        [Test]
+        public void TestEnumMeta()
+        {
+            var bin = _grid.GetBinary();
+
+            // Put to cache to populate metas
+            var binEnum = bin.ToBinary<IBinaryObject>(TestEnumRegistered.One);
+
+            Assert.AreEqual(_marsh.GetDescriptor(typeof (TestEnumRegistered)).TypeId, binEnum.GetBinaryType().TypeId);
+            Assert.AreEqual(0, binEnum.EnumValue);
+
+            var meta = binEnum.GetBinaryType();
+
+            Assert.IsTrue(meta.IsEnum);
+            Assert.AreEqual(typeof (TestEnumRegistered).Name, meta.TypeName);
+            Assert.AreEqual(0, meta.Fields.Count);
+        }
+
         /// <summary>
         /// Test arrays.
         /// </summary>
@@ -923,7 +933,7 @@ namespace Apache.Ignite.Core.Tests.Binary
 
             Assert.AreEqual(100, binObj.GetHashCode());
 
-            var binInArr = binObj.GetField<object[]>("inArr").Cast<IBinaryObject>().ToArray();
+            var binInArr = binObj.GetField<IBinaryObject[]>("inArr").ToArray();
 
             Assert.AreEqual(1, binInArr.Length);
             Assert.AreEqual(1, binInArr[0].GetField<int>("val"));
@@ -936,11 +946,11 @@ namespace Apache.Ignite.Core.Tests.Binary
 
             // 2. Test addition to array.
             binObj = _grid.GetBinary().GetBuilder(binObj).SetHashCode(200)
-                .SetField("inArr", new object[] { binInArr[0], null }).Build();
+                .SetField("inArr", new[] { binInArr[0], null }).Build();
 
             Assert.AreEqual(200, binObj.GetHashCode());
 
-            binInArr = binObj.GetField<object[]>("inArr").Cast<IBinaryObject>().ToArray();
+            binInArr = binObj.GetField<IBinaryObject[]>("inArr").ToArray();
 
             Assert.AreEqual(2, binInArr.Length);
             Assert.AreEqual(1, binInArr[0].GetField<int>("val"));
@@ -960,7 +970,7 @@ namespace Apache.Ignite.Core.Tests.Binary
 
             Assert.AreEqual(300, binObj.GetHashCode());
 
-            binInArr = binObj.GetField<object[]>("inArr").Cast<IBinaryObject>().ToArray();
+            binInArr = binObj.GetField<IBinaryObject[]>("inArr").ToArray();
 
             Assert.AreEqual(2, binInArr.Length);
             Assert.AreEqual(1, binInArr[0].GetField<int>("val"));
@@ -983,7 +993,7 @@ namespace Apache.Ignite.Core.Tests.Binary
 
             Assert.AreEqual(100, binObj.GetHashCode());
 
-            binInArr = binObj.GetField<object[]>("inArr").Cast<IBinaryObject>().ToArray();
+            binInArr = binObj.GetField<IBinaryObject[]>("inArr").ToArray();
 
             Assert.AreEqual(2, binInArr.Length);
             Assert.AreEqual(1, binInArr[0].GetField<int>("val"));
@@ -1003,7 +1013,7 @@ namespace Apache.Ignite.Core.Tests.Binary
 
             Assert.AreEqual(200, binObj.GetHashCode());
 
-            binInArr = binObj.GetField<object[]>("inArr").Cast<IBinaryObject>().ToArray();
+            binInArr = binObj.GetField<IBinaryObject[]>("inArr").ToArray();
 
             Assert.AreEqual(2, binInArr.Length);
             Assert.AreEqual(2, binInArr[0].GetField<int>("val"));
@@ -1031,7 +1041,7 @@ namespace Apache.Ignite.Core.Tests.Binary
 
             Assert.AreEqual(100, binObj.GetHashCode());
 
-            var binOutArr = binObj.GetField<object[]>("outArr").Cast<IBinaryObject>().ToArray();
+            var binOutArr = binObj.GetField<IBinaryObject[]>("outArr").ToArray();
 
             Assert.AreEqual(2, binOutArr.Length);
             Assert.AreEqual(1, binOutArr[0].GetField<IBinaryObject>("inner").GetField<int>("val"));
@@ -1052,7 +1062,7 @@ namespace Apache.Ignite.Core.Tests.Binary
 
             Assert.AreEqual(200, binObj.GetHashCode());
 
-            binInArr = binObj.GetField<object[]>("outArr").Cast<IBinaryObject>().ToArray();
+            binInArr = binObj.GetField<IBinaryObject[]>("outArr").ToArray();
 
             Assert.AreEqual(2, binInArr.Length);
             Assert.AreEqual(2, binOutArr[0].GetField<IBinaryObject>("inner").GetField<int>("val"));
@@ -1387,6 +1397,27 @@ namespace Apache.Ignite.Core.Tests.Binary
 
             Assert.AreEqual(new[] {"val", "valArr"}, decimalMeta.Fields);
         }
+
+        [Test]
+        public void TestBuildEnum()
+        {
+            var binary = _grid.GetBinary();
+
+            int val = (int) TestEnumRegistered.Two;
+
+            var binEnums = new[]
+            {
+                binary.BuildEnum(typeof (TestEnumRegistered), val),
+                binary.BuildEnum(typeof (TestEnumRegistered).Name, val)
+            };
+
+            foreach (var binEnum in binEnums)
+            {
+                Assert.IsTrue(binEnum.GetBinaryType().IsEnum);
+                Assert.AreEqual(val, binEnum.EnumValue);
+                Assert.AreEqual((TestEnumRegistered)val, binEnum.Deserialize<TestEnumRegistered>());
+            }
+        }
     }
 
     /// <summary>
@@ -1452,6 +1483,14 @@ namespace Apache.Ignite.Core.Tests.Binary
     }
 
     /// <summary>
+    /// Registered enumeration.
+    /// </summary>
+    public enum TestEnumRegistered
+    {
+        One, Two
+    }
+
+    /// <summary>
     /// binary with raw data.
     /// </summary>
     public class WithRaw : IBinarizable

http://git-wip-us.apache.org/repos/asf/ignite/blob/663e78dc/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 f7455be..88328ec 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinarySelfTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinarySelfTest.cs
@@ -25,6 +25,7 @@ namespace Apache.Ignite.Core.Tests.Binary
     using System;
     using System.Collections;
     using System.Collections.Generic;
+    using System.Diagnostics.CodeAnalysis;
     using System.Linq;
     using Apache.Ignite.Core.Binary;
     using Apache.Ignite.Core.Common;
@@ -479,11 +480,18 @@ namespace Apache.Ignite.Core.Tests.Binary
         * <summary>Check write of enum.</summary>
         */
         [Test]
+        [SuppressMessage("ReSharper", "ExpressionIsAlwaysNull")]
         public void TestWriteEnum()
         {
             TestEnum val = TestEnum.Val1;
 
             Assert.AreEqual(_marsh.Unmarshal<TestEnum>(_marsh.Marshal(val)), val);
+
+            TestEnum? val2 = TestEnum.Val1;
+            Assert.AreEqual(_marsh.Unmarshal<TestEnum?>(_marsh.Marshal(val2)), val2);
+
+            val2 = null;
+            Assert.AreEqual(_marsh.Unmarshal<TestEnum?>(_marsh.Marshal(val2)), val2);
         }
 
         /// <summary>
@@ -500,7 +508,13 @@ namespace Apache.Ignite.Core.Tests.Binary
 
             TestEnum val = TestEnum.Val1;
 
-            Assert.AreEqual(marsh.Unmarshal<TestEnum>(marsh.Marshal(val)), val);
+            var data = marsh.Marshal(val);
+
+            Assert.AreEqual(marsh.Unmarshal<TestEnum>(data), val);
+
+            var binEnum = marsh.Unmarshal<IBinaryObject>(data, true);
+
+            Assert.AreEqual(val, (TestEnum) binEnum.EnumValue);
         }
 
         /**
@@ -668,8 +682,13 @@ namespace Apache.Ignite.Core.Tests.Binary
 
             IBinaryObject portNewObj = marsh.Unmarshal<IBinaryObject>(data, BinaryMode.ForceBinary);
 
+            Assert.IsTrue(portNewObj.HasField("field1"));
+            Assert.IsTrue(portNewObj.HasField("field2"));
+            Assert.IsFalse(portNewObj.HasField("field3"));
+
             Assert.AreEqual(obj.Field1, portNewObj.GetField<int>("field1"));
             Assert.AreEqual(obj.Field2, portNewObj.GetField<int>("Field2"));
+            Assert.AreEqual(0, portNewObj.GetField<int>("field3"));
         }
 
         /**
@@ -881,8 +900,11 @@ namespace Apache.Ignite.Core.Tests.Binary
             Marshaller marsh =
                 new Marshaller(new BinaryConfiguration
                 {
-                    TypeConfigurations =
-                        new List<BinaryTypeConfiguration> {new BinaryTypeConfiguration(typeof (EnumType))}
+                    TypeConfigurations = new[]
+                    {
+                        new BinaryTypeConfiguration(typeof (EnumType)),
+                        new BinaryTypeConfiguration(typeof (TestEnum))
+                    }
                 });
 
             EnumType obj = new EnumType
@@ -897,6 +919,19 @@ namespace Apache.Ignite.Core.Tests.Binary
 
             Assert.AreEqual(obj.GetHashCode(), portObj.GetHashCode());
 
+            // Test enum field in binary form
+            var binEnum = portObj.GetField<IBinaryObject>("PEnum");
+            Assert.AreEqual(obj.PEnum.GetHashCode(), binEnum.GetHashCode());
+            Assert.AreEqual((int) obj.PEnum, binEnum.EnumValue);
+            Assert.AreEqual(obj.PEnum, binEnum.Deserialize<TestEnum>());
+            Assert.AreEqual(obj.PEnum, binEnum.Deserialize<object>());
+            Assert.AreEqual(typeof(TestEnum), binEnum.Deserialize<object>().GetType());
+            Assert.AreEqual(null, binEnum.GetField<object>("someField"));
+            Assert.IsFalse(binEnum.HasField("anyField"));
+
+            var binEnumArr = portObj.GetField<IBinaryObject[]>("PEnumArray");
+            Assert.IsTrue(binEnumArr.Select(x => x.Deserialize<TestEnum>()).SequenceEqual(obj.PEnumArray));
+
             EnumType newObj = portObj.Deserialize<EnumType>();
 
             Assert.AreEqual(obj.PEnum, newObj.PEnum);
@@ -1094,7 +1129,7 @@ namespace Apache.Ignite.Core.Tests.Binary
             inner.RawOuter = outer;
 
             var bytes = asbinary
-                ? marsh.Marshal(new IgniteBinary(marsh).ToBinary<IBinaryObject>(outer))
+                ? marsh.Marshal(new Binary(marsh).ToBinary<IBinaryObject>(outer))
                 : marsh.Marshal(outer);
 
             IBinaryObject outerObj;
@@ -2057,6 +2092,11 @@ namespace Apache.Ignite.Core.Tests.Binary
             Val1, Val2, Val3 = 10
         }
 
+        public enum TestEnum2
+        {
+            Val1, Val2, Val3 = 10
+        }
+
         public class DecimalReflective
         {
             /** */

http://git-wip-us.apache.org/repos/asf/ignite/blob/663e78dc/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs
index 64124d7..ce15739 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs
@@ -347,17 +347,16 @@ namespace Apache.Ignite.Core.Tests.Cache
             for (int i = 0; i < GridCount(); i++)
             {
                 var cache = Cache(i);
+                var entries = cache.Select(pair => pair.ToString() + GetKeyAffinity(cache, pair.Key)).ToArray();
 
-                if (!cache.IsEmpty())
-                {
-                    var entries = Enumerable.Range(0, 2000)
-                        .Select(x => new KeyValuePair<int, int>(x, cache.LocalPeek(x)))
-                        .Where(x => x.Value != 0)
-                        .Select(pair => pair.ToString() + GetKeyAffinity(cache, pair.Key))
-                        .Aggregate((acc, val) => string.Format("{0}, {1}", acc, val));
+                if (entries.Any())
+                    Assert.Fail("Cache '{0}' is not empty in grid [{1}]: ({2})", CacheName(), i,
+                        entries.Aggregate((acc, val) => string.Format("{0}, {1}", acc, val)));
 
-                    Assert.Fail("Cache '{0}' is not empty in grid [{1}]: ({2})", CacheName(), i, entries);
-                }
+                var size = cache.GetSize();
+                Assert.AreEqual(0, size,
+                    "Cache enumerator returned no entries, but cache '{0}' size is {1} in grid [{2}]",
+                    CacheName(), size, i);
             }
 
             Console.WriteLine("Test finished: " + TestContext.CurrentContext.Test.Name);

http://git-wip-us.apache.org/repos/asf/ignite/blob/663e78dc/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 1e999e3..87b7f9d 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTest.cs
@@ -899,6 +899,12 @@ namespace Apache.Ignite.Core.Tests.Compute
 
             var res = _grid1.GetCompute().ExecuteJavaTask<PlatformComputeEnum>(EchoTask, EchoTypeEnumField);
 
+            var enumMeta = _grid1.GetBinary().GetBinaryType(typeof (PlatformComputeEnum));
+
+            Assert.IsTrue(enumMeta.IsEnum);
+            Assert.AreEqual(enumMeta.TypeName, typeof(PlatformComputeEnum).Name);
+            Assert.AreEqual(0, enumMeta.Fields.Count);
+
             Assert.AreEqual(enumVal, res);
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/663e78dc/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 aa58054..3061773 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
@@ -64,7 +64,8 @@
                             <property name="typeName" value="org.apache.ignite.platform.PlatformComputeJavaBinarizable"/>
                         </bean>
                         <bean class="org.apache.ignite.binary.BinaryTypeConfiguration">
-                            <property name="typeName" value="org.apache.ignite.platform.PlatformComputeEnum"/>
+                            <property name="typeName" value="org.apache.ignite.platform.PlatformComputeEnum" />
+                            <property name="enum" value="true" />
                         </bean>
                     </list>
                 </property>

http://git-wip-us.apache.org/repos/asf/ignite/blob/663e78dc/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/native-client-test-cache-store.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/native-client-test-cache-store.xml b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/native-client-test-cache-store.xml
index b414a91..25a1d16 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/native-client-test-cache-store.xml
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/native-client-test-cache-store.xml
@@ -41,6 +41,7 @@
                     <property name="atomicityMode" value="TRANSACTIONAL"/>
                     <property name="writeThrough" value="true"/>
                     <property name="readThrough" value="true"/>
+                    <property name="keepBinaryInStore" value="true"/>
 
                     <property name="cacheStoreFactory">
                         <bean class="org.apache.ignite.platform.dotnet.PlatformDotNetCacheStoreFactory">

http://git-wip-us.apache.org/repos/asf/ignite/blob/663e78dc/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServiceProxyTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServiceProxyTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServiceProxyTest.cs
index 1797337..973381c 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServiceProxyTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServiceProxyTest.cs
@@ -48,7 +48,7 @@ namespace Apache.Ignite.Core.Tests.Services
         });
 
         /** */
-        protected readonly IIgniteBinary IgniteBinary;
+        protected readonly IBinary Binary;
 
         /** */
         private readonly PlatformMemoryManager _memory = new PlatformMemoryManager(1024);
@@ -64,7 +64,7 @@ namespace Apache.Ignite.Core.Tests.Services
         /// </summary>
         public ServiceProxyTest()
         {
-            IgniteBinary = new IgniteBinary(_marsh);
+            Binary = new Binary(_marsh);
         }
 
         /// <summary>
@@ -243,7 +243,7 @@ namespace Apache.Ignite.Core.Tests.Services
         /// </summary>
         protected T GetProxy<T>()
         {
-            _svc = new TestIgniteService(IgniteBinary);
+            _svc = new TestIgniteService(Binary);
 
             var prx = new ServiceProxy<T>(InvokeProxyMethod).GetTransparentProxy();
 
@@ -439,15 +439,15 @@ namespace Apache.Ignite.Core.Tests.Services
         private class TestIgniteService : ITestIgniteService, ITestIgniteServiceAmbiguity
         {
             /** */
-            private readonly IIgniteBinary _igniteBinary;
+            private readonly IBinary _binary;
 
             /// <summary>
             /// Initializes a new instance of the <see cref="TestIgniteService"/> class.
             /// </summary>
-            /// <param name="igniteBinary">Binary.</param>
-            public TestIgniteService(IIgniteBinary igniteBinary)
+            /// <param name="binary">Binary.</param>
+            public TestIgniteService(IBinary binary)
             {
-                _igniteBinary = igniteBinary;
+                _binary = binary;
             }
 
             /** <inheritdoc /> */
@@ -534,13 +534,13 @@ namespace Apache.Ignite.Core.Tests.Services
             /** <inheritdoc /> */
             public IBinaryObject BinarizableResultMethod(int arg1, TestBinarizableClass arg2)
             {
-                return _igniteBinary.ToBinary<IBinaryObject>(arg2);
+                return _binary.ToBinary<IBinaryObject>(arg2);
             }
 
             /** <inheritdoc /> */
             public IBinaryObject BinarizableArgAndResultMethod(int arg1, IBinaryObject arg2)
             {
-                return _igniteBinary.ToBinary<IBinaryObject>(arg2.Deserialize<TestBinarizableClass>());
+                return _binary.ToBinary<IBinaryObject>(arg2.Deserialize<TestBinarizableClass>());
             }
 
             /** <inheritdoc /> */
@@ -703,7 +703,7 @@ namespace Apache.Ignite.Core.Tests.Services
             var prx = GetProxy();
 
             var obj = new TestBinarizableClass { Prop = "PropValue" };
-            var portObj = IgniteBinary.ToBinary<IBinaryObject>(obj);
+            var portObj = Binary.ToBinary<IBinaryObject>(obj);
 
             var result = prx.BinarizableArgMethod(1, portObj);
 
@@ -731,7 +731,7 @@ namespace Apache.Ignite.Core.Tests.Services
             var prx = GetProxy();
             
             var obj = new TestBinarizableClass { Prop = "PropValue" };
-            var portObj = IgniteBinary.ToBinary<IBinaryObject>(obj);
+            var portObj = Binary.ToBinary<IBinaryObject>(obj);
 
             var result = prx.BinarizableArgAndResultMethod(1, portObj);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/663e78dc/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 ae10159..2c7d787 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
@@ -134,6 +134,7 @@
     <Compile Include="IgniteConfiguration.cs" />
     <Compile Include="Ignition.cs" />
     <Compile Include="IIgnite.cs" />
+    <Compile Include="Impl\Binary\BinaryEnum.cs" />
     <Compile Include="Impl\Cache\CacheAffinityImpl.cs" />
     <Compile Include="Impl\Cache\CacheEntry.cs" />
     <Compile Include="Impl\Cache\CacheEntryFilterHolder.cs" />
@@ -266,7 +267,7 @@
     <Compile Include="Impl\Binary\BinaryReader.cs" />
     <Compile Include="Impl\Binary\BinaryReflectiveActions.cs" />
     <Compile Include="Impl\Binary\BinaryReflectiveSerializer.cs" />
-    <Compile Include="Impl\Binary\IgniteBinary.cs" />
+    <Compile Include="Impl\Binary\Binary.cs" />
     <Compile Include="Impl\Binary\Structure\BinaryStructureTracker.cs" />
     <Compile Include="Impl\Binary\BinarySurrogateTypeDescriptor.cs" />
     <Compile Include="Impl\Binary\BinarySystemHandlers.cs" />
@@ -316,7 +317,7 @@
     <Compile Include="Binary\IBinaryRawReader.cs" />
     <Compile Include="Binary\IBinaryRawWriter.cs" />
     <Compile Include="Binary\IBinaryReader.cs" />
-    <Compile Include="Binary\IIgniteBinary.cs" />
+    <Compile Include="Binary\IBinary.cs" />
     <Compile Include="Binary\IBinarySerializer.cs" />
     <Compile Include="Binary\IBinaryWriter.cs" />
     <Compile Include="Binary\BinaryConfiguration.cs" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/663e78dc/modules/platforms/dotnet/Apache.Ignite.Core/Binary/BinaryTypeConfiguration.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Binary/BinaryTypeConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Binary/BinaryTypeConfiguration.cs
index 967aa52..99f8572 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Binary/BinaryTypeConfiguration.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Binary/BinaryTypeConfiguration.cs
@@ -48,6 +48,7 @@ namespace Apache.Ignite.Core.Binary
         public BinaryTypeConfiguration(Type type)
         {
             TypeName = type.AssemblyQualifiedName;
+            IsEnum = type.IsEnum;
         }
 
         /// <summary>
@@ -62,6 +63,7 @@ namespace Apache.Ignite.Core.Binary
             Serializer = cfg.Serializer;
             TypeName = cfg.TypeName;
             KeepDeserialized = cfg.KeepDeserialized;
+            IsEnum = cfg.IsEnum;
         }
 
         /// <summary>
@@ -101,6 +103,11 @@ namespace Apache.Ignite.Core.Binary
         public bool? KeepDeserialized { get; set; }
 
         /// <summary>
+        /// Gets or sets a value indicating whether this instance describes an enum type.
+        /// </summary>
+        public bool IsEnum { get; set; }
+
+        /// <summary>
         /// Returns a string that represents the current object.
         /// </summary>
         /// <returns>
@@ -108,9 +115,12 @@ namespace Apache.Ignite.Core.Binary
         /// </returns>
         public override string ToString()
         {
-            return typeof (BinaryTypeConfiguration).Name + " [TypeName=" + TypeName +
-                   ", NameMapper=" + NameMapper + ", IdMapper=" + IdMapper + ", Serializer=" + Serializer +
-                   ", AffinityKeyFieldName=" + AffinityKeyFieldName + ']';
+            return
+                string.Format(
+                    "{0} [TypeName={1}, NameMapper={2}, IdMapper={3}, Serializer={4}, AffinityKeyFieldName={5}, " +
+                    "KeepDeserialized={6}, IsEnum={7}]",
+                    typeof (BinaryTypeConfiguration).Name, TypeName, NameMapper, IdMapper, Serializer,
+                    AffinityKeyFieldName, KeepDeserialized, IsEnum);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/663e78dc/modules/platforms/dotnet/Apache.Ignite.Core/Binary/IBinary.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Binary/IBinary.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Binary/IBinary.cs
new file mode 100644
index 0000000..a0fc17f
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Binary/IBinary.cs
@@ -0,0 +1,136 @@
+/*
+ * 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.Binary
+{
+    using System;
+    using System.Collections.Generic;
+
+    /// <summary>
+    /// Defines binary objects functionality. With binary objects you are able to:
+    /// <list type="bullet">
+    ///     <item>
+    ///         <description>Seamlessly interoperate between Java, .NET, and C++.</description>
+    ///     </item>
+    ///     <item>
+    ///         <description>Make any object binary with zero code change to your existing code.</description>
+    ///     </item>
+    ///     <item>
+    ///         <description>Nest binary objects within each other.</description>
+    ///     </item>
+    ///     <item>
+    ///         <description>Automatically handle <c>circular</c> or <c>null</c> references.</description>
+    ///     </item>
+    ///     <item>
+    ///         <description>Automatically convert collections and maps between Java, .NET, and C++.</description>
+    ///     </item>
+    ///     <item>
+    ///         <description>Optionally avoid deserialization of objects on the server side.</description>
+    ///     </item>
+    ///     <item>
+    ///         <description>Avoid need to have concrete class definitions on the server side.</description>
+    ///     </item>
+    ///     <item>
+    ///         <description>Dynamically change structure of the classes without having to restart the cluster.</description>
+    ///     </item>
+    ///     <item>
+    ///         <description>Index into binary objects for querying purposes.</description>
+    ///     </item>
+    /// </list>
+    /// </summary>
+    public interface IBinary
+    {
+        /// <summary>
+        /// Converts provided object to binary form.
+        /// <para />
+        /// Note that object's type needs to be configured in <see cref="BinaryConfiguration"/>.
+        /// </summary>
+        /// <param name="obj">Object to convert.</param>
+        /// <returns>Converted object.</returns>
+        T ToBinary<T>(object obj);
+
+        /// <summary>
+        /// Create builder for the given binary object type. Note that this
+        /// type must be specified in <see cref="BinaryConfiguration"/>.
+        /// </summary>
+        /// <param name="type"></param>
+        /// <returns>Builder.</returns>
+        IBinaryObjectBuilder GetBuilder(Type type);
+
+        /// <summary>
+        /// Create builder for the given binary object type name. Note that this
+        /// type name must be specified in <see cref="BinaryConfiguration"/>.
+        /// </summary>
+        /// <param name="typeName">Type name.</param>
+        /// <returns>Builder.</returns>
+        IBinaryObjectBuilder GetBuilder(string typeName);
+
+        /// <summary>
+        /// Create builder over existing binary object.
+        /// </summary>
+        /// <param name="obj"></param>
+        /// <returns>Builder.</returns>
+        IBinaryObjectBuilder GetBuilder(IBinaryObject obj);
+
+        /// <summary>
+        /// Gets type id for the given type name.
+        /// </summary>
+        /// <param name="typeName">Type name.</param>
+        /// <returns>Type id.</returns>
+        int GetTypeId(string typeName);
+
+        /// <summary>
+        /// Gets metadata for all known types.
+        /// </summary>
+        /// <returns>Metadata.</returns>
+        ICollection<IBinaryType> GetBinaryTypes();
+
+        /// <summary>
+        /// Gets metadata for specified type id.
+        /// </summary>
+        /// <returns>Metadata.</returns>
+        IBinaryType GetBinaryType(int typeId);
+
+        /// <summary>
+        /// Gets metadata for specified type name.
+        /// </summary>
+        /// <returns>Metadata.</returns>
+        IBinaryType GetBinaryType(string typeName);
+
+        /// <summary>
+        /// Gets metadata for specified type.
+        /// </summary>
+        /// <returns>Metadata.</returns>
+        IBinaryType GetBinaryType(Type type);
+
+        /// <summary>
+        /// Converts enum to a binary form.
+        /// </summary>
+        /// <param name="typeName">Enum type name.</param>
+        /// <param name="value">Enum int value.</param>
+        /// <returns>Binary representation of the specified enum value.</returns>
+        IBinaryObject BuildEnum(string typeName, int value);
+
+        /// <summary>
+        /// Converts enum to a binary form.
+        /// </summary>
+        /// <param name="type">Enum type.</param>
+        /// <param name="value">Enum int value.</param>
+        /// <returns>Binary representation of the specified enum value.</returns>
+        IBinaryObject BuildEnum(Type type, int value);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/663e78dc/modules/platforms/dotnet/Apache.Ignite.Core/Binary/IBinaryObject.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Binary/IBinaryObject.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Binary/IBinaryObject.cs
index bd60e28..c5aa80e 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Binary/IBinaryObject.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Binary/IBinaryObject.cs
@@ -25,14 +25,6 @@ namespace Apache.Ignite.Core.Binary
     public interface IBinaryObject
     {
         /// <summary>
-        /// Gets binary object type ID.
-        /// </summary>
-        /// <value>
-        /// Type ID.
-        /// </value>
-        int TypeId { get; }
-
-        /// <summary>
         /// Gets object metadata.
         /// </summary>
         /// <returns>Metadata.</returns>
@@ -50,11 +42,26 @@ namespace Apache.Ignite.Core.Binary
         TF GetField<TF>(string fieldName);
 
         /// <summary>
+        /// Determines whether the field with specified name exists in this instance.
+        /// </summary>
+        /// <param name="fieldName">Name of the field.</param>
+        /// <returns>True if there is a field with specified name; false otherwise.</returns>
+        bool HasField(string fieldName);
+
+        /// <summary>
         /// Gets fully deserialized instance of binary object.
         /// </summary>
         /// <returns>
         /// Fully deserialized instance of binary object.
         /// </returns>
         T Deserialize<T>();
+
+        /// <summary>
+        /// Gets the value of underlying enum in int form.
+        /// </summary>
+        /// <value>
+        /// The value of underlying enum in int form.
+        /// </value>
+        int EnumValue { get; }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/663e78dc/modules/platforms/dotnet/Apache.Ignite.Core/Binary/IBinaryType.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Binary/IBinaryType.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Binary/IBinaryType.cs
index 7b34e07..bec863f 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Binary/IBinaryType.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Binary/IBinaryType.cs
@@ -48,5 +48,18 @@ namespace Apache.Ignite.Core.Binary
         /// </summary>
         /// <returns>Affinity key field name or null in case it is not provided.</returns>
         string AffinityKeyFieldName { get; }
+
+        /// <summary>
+        /// Gets a value indicating whether this type represents an enum.
+        /// </summary>
+        /// <value>   
+        /// <c>true</c> if this instance represents an enum; otherwise, <c>false</c>.
+        /// </value>
+        bool IsEnum { get; }
+
+        /// <summary>
+        /// Gets the type identifier.
+        /// </summary>
+        int TypeId { get; }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/663e78dc/modules/platforms/dotnet/Apache.Ignite.Core/Binary/IIgniteBinary.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Binary/IIgniteBinary.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Binary/IIgniteBinary.cs
deleted file mode 100644
index 25ea981..0000000
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Binary/IIgniteBinary.cs
+++ /dev/null
@@ -1,120 +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.
- */
-
-namespace Apache.Ignite.Core.Binary
-{
-    using System;
-    using System.Collections.Generic;
-
-    /// <summary>
-    /// Defines binary objects functionality. With binary objects you are able to:
-    /// <list type="bullet">
-    ///     <item>
-    ///         <description>Seamlessly interoperate between Java, .NET, and C++.</description>
-    ///     </item>
-    ///     <item>
-    ///         <description>Make any object binary with zero code change to your existing code.</description>
-    ///     </item>
-    ///     <item>
-    ///         <description>Nest binary objects within each other.</description>
-    ///     </item>
-    ///     <item>
-    ///         <description>Automatically handle <c>circular</c> or <c>null</c> references.</description>
-    ///     </item>
-    ///     <item>
-    ///         <description>Automatically convert collections and maps between Java, .NET, and C++.</description>
-    ///     </item>
-    ///     <item>
-    ///         <description>Optionally avoid deserialization of objects on the server side.</description>
-    ///     </item>
-    ///     <item>
-    ///         <description>Avoid need to have concrete class definitions on the server side.</description>
-    ///     </item>
-    ///     <item>
-    ///         <description>Dynamically change structure of the classes without having to restart the cluster.</description>
-    ///     </item>
-    ///     <item>
-    ///         <description>Index into binary objects for querying purposes.</description>
-    ///     </item>
-    /// </list>
-    /// </summary>
-    public interface IIgniteBinary
-    {
-        /// <summary>
-        /// Converts provided object to binary form.
-        /// <para />
-        /// Note that object's type needs to be configured in <see cref="BinaryConfiguration"/>.
-        /// </summary>
-        /// <param name="obj">Object to convert.</param>
-        /// <returns>Converted object.</returns>
-        T ToBinary<T>(object obj);
-
-        /// <summary>
-        /// Create builder for the given binary object type. Note that this
-        /// type must be specified in <see cref="BinaryConfiguration"/>.
-        /// </summary>
-        /// <param name="type"></param>
-        /// <returns>Builder.</returns>
-        IBinaryObjectBuilder GetBuilder(Type type);
-
-        /// <summary>
-        /// Create builder for the given binary object type name. Note that this
-        /// type name must be specified in <see cref="BinaryConfiguration"/>.
-        /// </summary>
-        /// <param name="typeName">Type name.</param>
-        /// <returns>Builder.</returns>
-        IBinaryObjectBuilder GetBuilder(string typeName);
-
-        /// <summary>
-        /// Create builder over existing binary object.
-        /// </summary>
-        /// <param name="obj"></param>
-        /// <returns>Builder.</returns>
-        IBinaryObjectBuilder GetBuilder(IBinaryObject obj);
-
-        /// <summary>
-        /// Gets type id for the given type name.
-        /// </summary>
-        /// <param name="typeName">Type name.</param>
-        /// <returns>Type id.</returns>
-        int GetTypeId(string typeName);
-
-        /// <summary>
-        /// Gets metadata for all known types.
-        /// </summary>
-        /// <returns>Metadata.</returns>
-        ICollection<IBinaryType> GetBinaryTypes();
-
-        /// <summary>
-        /// Gets metadata for specified type id.
-        /// </summary>
-        /// <returns>Metadata.</returns>
-        IBinaryType GetBinaryType(int typeId);
-
-        /// <summary>
-        /// Gets metadata for specified type name.
-        /// </summary>
-        /// <returns>Metadata.</returns>
-        IBinaryType GetBinaryType(string typeName);
-
-        /// <summary>
-        /// Gets metadata for specified type.
-        /// </summary>
-        /// <returns>Metadata.</returns>
-        IBinaryType GetBinaryType(Type type);
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/663e78dc/modules/platforms/dotnet/Apache.Ignite.Core/IIgnite.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IIgnite.cs b/modules/platforms/dotnet/Apache.Ignite.Core/IIgnite.cs
index b9d9555..08e5f6b 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/IIgnite.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/IIgnite.cs
@@ -111,11 +111,11 @@ namespace Apache.Ignite.Core
         IDataStreamer<TK, TV> GetDataStreamer<TK, TV>(string cacheName);
 
         /// <summary>
-        /// Gets an instance of <see cref="IIgniteBinary"/> interface.
+        /// Gets an instance of <see cref="IBinary"/> interface.
         /// </summary>
-        /// <returns>Instance of <see cref="IIgniteBinary"/> interface</returns>
+        /// <returns>Instance of <see cref="IBinary"/> interface</returns>
         [SuppressMessage("Microsoft.Design", "CA1024:UsePropertiesWhereAppropriate", Justification = "Semantics.")]
-        IIgniteBinary GetBinary();
+        IBinary GetBinary();
 
         /// <summary>
         /// Gets affinity service to provide information about data partitioning and distribution.

http://git-wip-us.apache.org/repos/asf/ignite/blob/663e78dc/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
new file mode 100644
index 0000000..43a4bb8
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Binary.cs
@@ -0,0 +1,216 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.Core.Impl.Binary
+{
+    using System;
+    using System.Collections.Generic;
+    using System.IO;
+    using Apache.Ignite.Core.Binary;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Impl.Binary.IO;
+    using Apache.Ignite.Core.Impl.Binary.Metadata;
+    using Apache.Ignite.Core.Impl.Common;
+
+    /// <summary>
+    /// Binary implementation.
+    /// </summary>
+    internal class Binary : IBinary
+    {
+        /** Owning grid. */
+        private readonly Marshaller _marsh;
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="marsh">Marshaller.</param>
+        internal Binary(Marshaller marsh)
+        {
+            _marsh = marsh;
+        }
+
+        /** <inheritDoc /> */
+        public T ToBinary<T>(object obj)
+        {
+            if (obj is IBinaryObject)
+                return (T)obj;
+
+            IBinaryStream stream = new BinaryHeapStream(1024);
+
+            // Serialize.
+            BinaryWriter writer = _marsh.StartMarshal(stream);
+
+            try
+            {
+                writer.Write(obj);
+            }
+            finally
+            {
+                // Save metadata.
+                _marsh.FinishMarshal(writer);
+            }
+
+            // Deserialize.
+            stream.Seek(0, SeekOrigin.Begin);
+
+            return _marsh.Unmarshal<T>(stream, BinaryMode.ForceBinary);
+        }
+
+        /** <inheritDoc /> */
+        public IBinaryObjectBuilder GetBuilder(Type type)
+        {
+            IgniteArgumentCheck.NotNull(type, "type");
+
+            IBinaryTypeDescriptor desc = _marsh.GetDescriptor(type);
+
+            if (desc == null)
+                throw new IgniteException("Type is not binary (add it to BinaryConfiguration): " + 
+                    type.FullName);
+
+            return Builder0(null, BinaryFromDescriptor(desc), desc);
+        }
+
+        /** <inheritDoc /> */
+        public IBinaryObjectBuilder GetBuilder(string typeName)
+        {
+            IgniteArgumentCheck.NotNullOrEmpty(typeName, "typeName");
+
+            IBinaryTypeDescriptor desc = _marsh.GetDescriptor(typeName);
+            
+            return Builder0(null, BinaryFromDescriptor(desc), desc);
+        }
+
+        /** <inheritDoc /> */
+        public IBinaryObjectBuilder GetBuilder(IBinaryObject obj)
+        {
+            IgniteArgumentCheck.NotNull(obj, "obj");
+
+            BinaryObject obj0 = obj as BinaryObject;
+
+            if (obj0 == null)
+                throw new ArgumentException("Unsupported object type: " + obj.GetType());
+
+            IBinaryTypeDescriptor desc = _marsh.GetDescriptor(true, obj0.TypeId);
+            
+            return Builder0(null, obj0, desc);
+        }
+
+        /** <inheritDoc /> */
+        public int GetTypeId(string typeName)
+        {
+            IgniteArgumentCheck.NotNullOrEmpty(typeName, "typeName");
+
+            return Marshaller.GetDescriptor(typeName).TypeId;
+        }
+
+        /** <inheritDoc /> */
+        public ICollection<IBinaryType> GetBinaryTypes()
+        {
+            return Marshaller.Ignite.ClusterGroup.GetBinaryTypes();
+        }
+
+        /** <inheritDoc /> */
+        public IBinaryType GetBinaryType(int typeId)
+        {
+            return Marshaller.GetBinaryType(typeId);
+        }
+
+        /** <inheritDoc /> */
+        public IBinaryType GetBinaryType(string typeName)
+        {
+            IgniteArgumentCheck.NotNullOrEmpty(typeName, "typeName");
+
+            return GetBinaryType(GetTypeId(typeName));
+        }
+
+        /** <inheritDoc /> */
+        public IBinaryType GetBinaryType(Type type)
+        {
+            IgniteArgumentCheck.NotNull(type, "type");
+
+            var desc = Marshaller.GetDescriptor(type);
+
+            return desc == null ? null : Marshaller.GetBinaryType(desc.TypeId);
+        }
+
+        /** <inheritDoc /> */
+        public IBinaryObject BuildEnum(string typeName, int value)
+        {
+            IgniteArgumentCheck.NotNullOrEmpty(typeName, "typeName");
+
+            var desc = Marshaller.GetDescriptor(typeName);
+
+            IgniteArgumentCheck.Ensure(desc.IsEnum, "typeName", "Type should be an Enum.");
+
+            _marsh.PutBinaryType(desc);
+
+            return new BinaryEnum(GetTypeId(typeName), value, Marshaller);
+        }
+
+        /** <inheritDoc /> */
+        public IBinaryObject BuildEnum(Type type, int value)
+        {
+            IgniteArgumentCheck.NotNull(type, "type");
+            IgniteArgumentCheck.Ensure(type.IsEnum, "type", "Type should be an Enum.");
+
+            return BuildEnum(type.Name, value);
+        }
+
+        /// <summary>
+        /// Marshaller.
+        /// </summary>
+        internal Marshaller Marshaller
+        {
+            get
+            {
+                return _marsh;
+            }
+        }
+
+        /// <summary>
+        /// Create empty binary object from descriptor.
+        /// </summary>
+        /// <param name="desc">Descriptor.</param>
+        /// <returns>Empty binary object.</returns>
+        private BinaryObject BinaryFromDescriptor(IBinaryTypeDescriptor desc)
+        {
+            var len = BinaryObjectHeader.Size;
+
+            var hdr = new BinaryObjectHeader(desc.TypeId, 0, len, 0, len,
+                desc.UserType ? BinaryObjectHeader.Flag.UserType : BinaryObjectHeader.Flag.None);
+
+            var stream = new BinaryHeapStream(len);
+
+            BinaryObjectHeader.Write(hdr, stream, 0);
+
+            return new BinaryObject(_marsh, stream.InternalArray, 0, hdr);
+        }
+
+        /// <summary>
+        /// Internal builder creation routine.
+        /// </summary>
+        /// <param name="parent">Parent builder.</param>
+        /// <param name="obj">binary object.</param>
+        /// <param name="desc">Type descriptor.</param>
+        /// <returns>Builder.</returns>
+        private BinaryObjectBuilder Builder0(BinaryObjectBuilder parent, BinaryObject obj, 
+            IBinaryTypeDescriptor desc)
+        {
+            return new BinaryObjectBuilder(this, parent, obj, desc);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/663e78dc/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryEnum.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryEnum.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryEnum.cs
new file mode 100644
index 0000000..97f44b0
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryEnum.cs
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.Core.Impl.Binary
+{
+    using System;
+    using System.Diagnostics;
+    using Apache.Ignite.Core.Binary;
+
+    /// <summary>
+    /// Represents a typed enum in binary form.
+    /// </summary>
+    internal class BinaryEnum : IBinaryObject, IEquatable<BinaryEnum>
+    {
+        /** Type id. */
+        private readonly int _typeId;
+
+        /** Value. */
+        private readonly int _enumValue;
+
+        /** Marshaller. */
+        private readonly Marshaller _marsh;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="BinaryEnum" /> class.
+        /// </summary>
+        /// <param name="typeId">The type identifier.</param>
+        /// <param name="enumValue">The value.</param>
+        /// <param name="marsh">The marshaller.</param>
+        public BinaryEnum(int typeId, int enumValue, Marshaller marsh)
+        {
+            Debug.Assert(marsh != null);
+
+            _typeId = typeId;
+            _enumValue = enumValue;
+            _marsh = marsh;
+        }
+
+        /** <inheritdoc /> */
+        public int TypeId
+        {
+            get { return _typeId; }
+        }
+
+        /** <inheritdoc /> */
+        public IBinaryType GetBinaryType()
+        {
+            return _marsh.GetBinaryType(_typeId);
+        }
+
+        /** <inheritdoc /> */
+        public TF GetField<TF>(string fieldName)
+        {
+            return default(TF);
+        }
+
+        /** <inheritdoc /> */
+        public bool HasField(string fieldName)
+        {
+            return false;
+        }
+
+        /** <inheritdoc /> */
+        public T Deserialize<T>()
+        {
+            return BinaryUtils.GetEnumValue<T>(_enumValue, _typeId, _marsh);
+        }
+
+        /** <inheritdoc /> */
+        public int EnumValue
+        {
+            get { return _enumValue; }
+        }
+
+        /** <inheritdoc /> */
+        public bool Equals(BinaryEnum other)
+        {
+            if (ReferenceEquals(null, other))
+                return false;
+
+            if (ReferenceEquals(this, other))
+                return true;
+
+            return _typeId == other._typeId && _enumValue == other._enumValue;
+        }
+
+        /** <inheritdoc /> */
+        public override bool Equals(object obj)
+        {
+            if (ReferenceEquals(null, obj))
+                return false;
+
+            if (ReferenceEquals(this, obj))
+                return true;
+
+            if (obj.GetType() != GetType())
+                return false;
+
+            return Equals((BinaryEnum) obj);
+        }
+
+        /** <inheritdoc /> */
+        public override int GetHashCode()
+        {
+            return _enumValue.GetHashCode();
+        }
+
+        /** <inheritdoc /> */
+        public static bool operator ==(BinaryEnum left, BinaryEnum right)
+        {
+            return Equals(left, right);
+        }
+
+        /** <inheritdoc /> */
+        public static bool operator !=(BinaryEnum left, BinaryEnum right)
+        {
+            return !Equals(left, right);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/663e78dc/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryFullTypeDescriptor.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryFullTypeDescriptor.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryFullTypeDescriptor.cs
index 9649595..3d2b34d 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryFullTypeDescriptor.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryFullTypeDescriptor.cs
@@ -63,6 +63,9 @@ namespace Apache.Ignite.Core.Impl.Binary
         /** Type schema. */
         private readonly BinaryObjectSchema _schema = new BinaryObjectSchema();
 
+        /** Enum flag. */
+        private bool _isEnum;
+
         /// <summary>
         /// Constructor.
         /// </summary>
@@ -75,6 +78,7 @@ namespace Apache.Ignite.Core.Impl.Binary
         /// <param name="serializer">Serializer.</param>
         /// <param name="keepDeserialized">Whether to cache deserialized value in IBinaryObject</param>
         /// <param name="affKeyFieldName">Affinity field key name.</param>
+        /// <param name="isEnum">Enum flag.</param>
         public BinaryFullTypeDescriptor(
             Type type, 
             int typeId, 
@@ -84,7 +88,8 @@ namespace Apache.Ignite.Core.Impl.Binary
             IBinaryIdMapper idMapper, 
             IBinarySerializer serializer, 
             bool keepDeserialized, 
-            string affKeyFieldName)
+            string affKeyFieldName,
+            bool isEnum)
         {
             _type = type;
             _typeId = typeId;
@@ -95,6 +100,7 @@ namespace Apache.Ignite.Core.Impl.Binary
             _serializer = serializer;
             _keepDeserialized = keepDeserialized;
             _affKeyFieldName = affKeyFieldName;
+            _isEnum = isEnum;
         }
 
         /// <summary>
@@ -169,6 +175,12 @@ namespace Apache.Ignite.Core.Impl.Binary
             get { return _affKeyFieldName; }
         }
 
+        /** <inheritdoc/> */
+        public bool IsEnum
+        {
+            get { return _isEnum; }
+        }
+
         /** <inheritDoc /> */
         public BinaryStructure WriterTypeStructure
         {

http://git-wip-us.apache.org/repos/asf/ignite/blob/663e78dc/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 fd60da7..90607dd 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObject.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObject.cs
@@ -17,14 +17,17 @@
 
 namespace Apache.Ignite.Core.Impl.Binary
 {
+    using System;
     using System.Collections;
     using System.Collections.Generic;
+    using System.Diagnostics;
     using System.IO;
     using System.Runtime.CompilerServices;
     using System.Text;
     using Apache.Ignite.Core.Binary;
     using Apache.Ignite.Core.Common;
     using Apache.Ignite.Core.Impl.Binary.IO;
+    using Apache.Ignite.Core.Impl.Common;
 
     /// <summary>
     /// Binary object.
@@ -61,6 +64,10 @@ namespace Apache.Ignite.Core.Impl.Binary
         /// <param name="header">The header.</param>
         public BinaryObject(Marshaller marsh, byte[] data, int offset, BinaryObjectHeader header)
         {
+            Debug.Assert(marsh != null);
+            Debug.Assert(data != null);
+            Debug.Assert(offset >= 0 && offset < data.Length);
+
             _marsh = marsh;
 
             _data = data;
@@ -78,11 +85,23 @@ namespace Apache.Ignite.Core.Impl.Binary
         /** <inheritdoc /> */
         public T GetField<T>(string fieldName)
         {
+            IgniteArgumentCheck.NotNullOrEmpty(fieldName, "fieldName");
+
             int pos;
 
             return TryGetFieldPosition(fieldName, out pos) ? GetField<T>(pos, null) : default(T);
         }
 
+        /** <inheritdoc /> */
+        public bool HasField(string fieldName)
+        {
+            IgniteArgumentCheck.NotNullOrEmpty(fieldName, "fieldName");
+
+            int pos;
+
+            return TryGetFieldPosition(fieldName, out pos);
+        }
+
         /// <summary>
         /// Gets field value on the given object.
         /// </summary>
@@ -104,6 +123,16 @@ namespace Apache.Ignite.Core.Impl.Binary
             return Deserialize<T>(BinaryMode.Deserialize);
         }
 
+        /** <inheritdoc /> */
+        public int EnumValue
+        {
+            get
+            {
+                throw new NotSupportedException("IBinaryObject.Value is only supported for enums. " +
+                    "Check IBinaryObject.IsEnum property before accessing Value.");
+            }
+        }
+
         /// <summary>
         /// Internal deserialization routine.
         /// </summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/663e78dc/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 97cc381..7ef6259 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectBuilder.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectBuilder.cs
@@ -37,7 +37,7 @@ namespace Apache.Ignite.Core.Impl.Binary
             new Dictionary<int, BinaryBuilderField>();
         
         /** Binary. */
-        private readonly IgniteBinary _igniteBinary;
+        private readonly Binary _binary;
 
         /** */
         private readonly BinaryObjectBuilder _parent;
@@ -79,18 +79,18 @@ namespace Apache.Ignite.Core.Impl.Binary
         /// <summary>
         /// Constructor.
         /// </summary>
-        /// <param name="igniteBinary">Binary.</param>
+        /// <param name="binary">Binary.</param>
         /// <param name="parent">Parent builder.</param>
         /// <param name="obj">Initial binary object.</param>
         /// <param name="desc">Type descriptor.</param>
-        public BinaryObjectBuilder(IgniteBinary igniteBinary, BinaryObjectBuilder parent, 
+        public BinaryObjectBuilder(Binary binary, BinaryObjectBuilder parent, 
             BinaryObject obj, IBinaryTypeDescriptor desc)
         {
-            Debug.Assert(igniteBinary != null);
+            Debug.Assert(binary != null);
             Debug.Assert(obj != null);
             Debug.Assert(desc != null);
 
-            _igniteBinary = igniteBinary;
+            _binary = binary;
             _parent = parent ?? this;
             _obj = obj;
             _desc = desc;
@@ -361,7 +361,7 @@ namespace Apache.Ignite.Core.Impl.Binary
 
             BinaryHeapStream outStream = new BinaryHeapStream(estimatedCapacity);
 
-            BinaryWriter writer = _igniteBinary.Marshaller.StartMarshal(outStream);
+            BinaryWriter writer = _binary.Marshaller.StartMarshal(outStream);
 
             writer.SetBuilder(this);
 
@@ -374,10 +374,10 @@ namespace Apache.Ignite.Core.Impl.Binary
                 writer.Write(this);
                 
                 // Process metadata.
-                _igniteBinary.Marshaller.FinishMarshal(writer);
+                _binary.Marshaller.FinishMarshal(writer);
 
                 // Create binary object once metadata is processed.
-                return new BinaryObject(_igniteBinary.Marshaller, outStream.InternalArray, 0, 
+                return new BinaryObject(_binary.Marshaller, outStream.InternalArray, 0, 
                     BinaryObjectHeader.Read(outStream, 0));
             }
             finally
@@ -394,9 +394,9 @@ namespace Apache.Ignite.Core.Impl.Binary
         /// <returns>Child builder.</returns>
         public BinaryObjectBuilder Child(BinaryObject obj)
         {
-            var desc = _igniteBinary.Marshaller.GetDescriptor(true, obj.TypeId);
+            var desc = _binary.Marshaller.GetDescriptor(true, obj.TypeId);
 
-            return new BinaryObjectBuilder(_igniteBinary, null, obj, desc);
+            return new BinaryObjectBuilder(_binary, null, obj, desc);
         }
         
         /// <summary>
@@ -436,7 +436,7 @@ namespace Apache.Ignite.Core.Impl.Binary
 
             var hdr = _obj.Data[pos];
 
-            var field = new BinaryBuilderField(typeof(T), val, hdr, GetWriteAction(hdr));
+            var field = new BinaryBuilderField(typeof(T), val, hdr, GetWriteAction(hdr, pos));
             
             _parent._cache[pos] = field;
 
@@ -447,8 +447,9 @@ namespace Apache.Ignite.Core.Impl.Binary
         /// Gets the write action by header.
         /// </summary>
         /// <param name="header">The header.</param>
+        /// <param name="pos">Position.</param>
         /// <returns>Write action.</returns>
-        private static Action<BinaryWriter, object> GetWriteAction(byte header)
+        private Action<BinaryWriter, object> GetWriteAction(byte header, int pos)
         {
             // We need special actions for all cases where SetField(X) produces different result from SetSpecialField(X)
             // Arrays, Collections, Dates
@@ -466,9 +467,20 @@ namespace Apache.Ignite.Core.Impl.Binary
 
                 case BinaryUtils.TypeArrayTimestamp:
                     return WriteTimestampArrayAction;
-            }
 
-            return null;
+                case BinaryUtils.TypeArrayEnum:
+                    using (var stream = new BinaryHeapStream(_obj.Data))
+                    {
+                        stream.Seek(pos, SeekOrigin.Begin + 1);
+
+                        var elementTypeId = stream.ReadInt();
+
+                        return (w, o) => w.WriteEnumArrayInternal((Array) o, elementTypeId);
+                    }
+
+                default:
+                    return null;
+            }
         }
 
         /// <summary>
@@ -510,7 +522,7 @@ namespace Apache.Ignite.Core.Impl.Binary
             try
             {
                 // Prepare fields.
-                IBinaryTypeHandler metaHnd = _igniteBinary.Marshaller.GetBinaryTypeHandler(desc);
+                IBinaryTypeHandler metaHnd = _binary.Marshaller.GetBinaryTypeHandler(desc);
 
                 IDictionary<int, BinaryBuilderField> vals0;
 
@@ -546,7 +558,7 @@ namespace Apache.Ignite.Core.Impl.Binary
                     IDictionary<string, int> meta = metaHnd.OnObjectWriteFinished();
 
                     if (meta != null)
-                        _parent._ctx.Writer.SaveMetadata(desc.TypeId, desc.TypeName, desc.AffinityKeyFieldName, meta);
+                        _parent._ctx.Writer.SaveMetadata(desc, meta);
                 }
             }
             finally

http://git-wip-us.apache.org/repos/asf/ignite/blob/663e78dc/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 9aeb908..7b887a9 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReader.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReader.cs
@@ -399,13 +399,34 @@ namespace Apache.Ignite.Core.Impl.Binary
         /** <inheritdoc /> */
         public T ReadEnum<T>(string fieldName)
         {
-            return ReadField(fieldName, BinaryUtils.ReadEnum<T>, BinaryUtils.TypeEnum);
+            return SeekField(fieldName) ? ReadEnum<T>() : default(T);
         }
 
         /** <inheritdoc /> */
         public T ReadEnum<T>()
         {
-            return Read(BinaryUtils.ReadEnum<T>, BinaryUtils.TypeEnum);
+            var hdr = ReadByte();
+
+            switch (hdr)
+            {
+                case BinaryUtils.HdrNull:
+                    return default(T);
+
+                case BinaryUtils.TypeEnum:
+                    // Never read enums in binary mode when reading a field (we do not support half-binary objects)
+                    return ReadEnum0<T>(this, false);  
+
+                case BinaryUtils.HdrFull:
+                    // Unregistered enum written as serializable
+                    Stream.Seek(-1, SeekOrigin.Current);
+
+                    return ReadObject<T>(); 
+
+                default:
+                    throw new BinaryObjectException(
+                        string.Format("Invalid header on enum deserialization. Expected: {0} or {1} but was: {2}",
+                            BinaryUtils.TypeEnum, BinaryUtils.HdrFull, hdr));
+            }
         }
 
         /** <inheritdoc /> */
@@ -560,15 +581,16 @@ namespace Apache.Ignite.Core.Impl.Binary
                     res = ReadBinaryObject<T>(doDetach);
 
                     return true;
-            }
 
-            if (BinaryUtils.IsPredefinedType(hdr))
-            {
-                res = BinarySystemHandlers.ReadSystemType<T>(hdr, this);
+                case BinaryUtils.TypeEnum:
+                    res = ReadEnum0<T>(this, _mode != BinaryMode.Deserialize);
 
-                return true;
+                    return true;
             }
 
+            if (BinarySystemHandlers.TryReadSystemType(hdr, this, out res))
+                return true;
+
             throw new BinaryObjectException("Invalid header on deserialization [pos=" + pos + ", hdr=" + hdr + ']');
         }
 
@@ -961,5 +983,20 @@ namespace Apache.Ignite.Core.Impl.Binary
         {
             return IsNotNullHeader(expHdr) ? readFunc(Stream) : default(T);
         }
+
+        /// <summary>
+        /// Reads the enum.
+        /// </summary>
+        private static T ReadEnum0<T>(BinaryReader reader, bool keepBinary)
+        {
+            var enumType = reader.ReadInt();
+
+            var enumValue = reader.ReadInt();
+
+            if (!keepBinary)
+                return BinaryUtils.GetEnumValue<T>(enumValue, enumType, reader.Marshaller);
+
+            return TypeCaster<T>.Cast(new BinaryEnum(enumType, enumValue, reader.Marshaller));
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/663e78dc/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinarySurrogateTypeDescriptor.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinarySurrogateTypeDescriptor.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinarySurrogateTypeDescriptor.cs
index 247b40d..04028b5 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinarySurrogateTypeDescriptor.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinarySurrogateTypeDescriptor.cs
@@ -124,6 +124,12 @@ namespace Apache.Ignite.Core.Impl.Binary
             get { return null; }
         }
 
+        /** <inheritdoc/> */
+        public bool IsEnum
+        {
+            get { return false; }
+        }
+
         /** <inheritDoc /> */
         public BinaryStructure WriterTypeStructure
         {

http://git-wip-us.apache.org/repos/asf/ignite/blob/663e78dc/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinarySystemHandlers.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinarySystemHandlers.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinarySystemHandlers.cs
index b49c29d..0af1e82 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinarySystemHandlers.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinarySystemHandlers.cs
@@ -22,6 +22,7 @@ namespace Apache.Ignite.Core.Impl.Binary
     using System.Collections.Generic;
     using System.Diagnostics;
     using System.Diagnostics.CodeAnalysis;
+    using Apache.Ignite.Core.Binary;
     using Apache.Ignite.Core.Impl.Binary.IO;
     using Apache.Ignite.Core.Impl.Common;
 
@@ -166,7 +167,6 @@ namespace Apache.Ignite.Core.Impl.Binary
             ReadHandlers[BinaryUtils.TypeMapEntry] = new BinarySystemReader(ReadMapEntry);
             
             // 16. Enum.
-            ReadHandlers[BinaryUtils.TypeEnum] = new BinarySystemReader<int>(BinaryUtils.ReadEnum<int>);
             ReadHandlers[BinaryUtils.TypeArrayEnum] = new BinarySystemReader(ReadEnumArray);
         }
 
@@ -212,6 +212,8 @@ namespace Apache.Ignite.Core.Impl.Binary
                 return WriteGuid;
             if (type == typeof (BinaryObject))
                 return WriteBinary;
+            if (type == typeof (BinaryEnum))
+                return WriteBinaryEnum;
             if (type == typeof (ArrayList))
                 return WriteArrayList;
             if (type == typeof(Hashtable))
@@ -257,11 +259,11 @@ namespace Apache.Ignite.Core.Impl.Binary
                 if (elemType == typeof(Guid?))
                     return WriteGuidArray;
                 // Enums.
-                if (elemType.IsEnum)
+                if (elemType.IsEnum || elemType == typeof(BinaryEnum))
                     return WriteEnumArray;
                 
                 // Object array.
-                if (elemType == typeof (object))
+                if (elemType == typeof (object) || elemType == typeof(IBinaryObject) || elemType == typeof(BinaryObject))
                     return WriteArray;
             }
 
@@ -329,13 +331,18 @@ namespace Apache.Ignite.Core.Impl.Binary
         /// <summary>
         /// Reads an object of predefined type.
         /// </summary>
-        public static T ReadSystemType<T>(byte typeId, BinaryReader ctx)
+        public static bool TryReadSystemType<T>(byte typeId, BinaryReader ctx, out T res)
         {
             var handler = ReadHandlers[typeId];
 
-            Debug.Assert(handler != null, "Cannot find predefined read handler: " + typeId);
-            
-            return handler.Read<T>(ctx);
+            if (handler == null)
+            {
+                res = default(T);
+                return false;
+            }
+
+            res = handler.Read<T>(ctx);
+            return true;
         }
         
         /// <summary>
@@ -629,9 +636,20 @@ namespace Apache.Ignite.Core.Impl.Binary
         /// </summary>
         private static void WriteEnum(BinaryWriter ctx, object obj)
         {
+            ctx.WriteEnum(obj);
+        }
+
+        /// <summary>
+        /// Write enum.
+        /// </summary>
+        private static void WriteBinaryEnum(BinaryWriter ctx, object obj)
+        {
+            var binEnum = (BinaryEnum) obj;
+
             ctx.Stream.WriteByte(BinaryUtils.TypeEnum);
 
-            BinaryUtils.WriteEnum(ctx, obj);
+            ctx.WriteInt(binEnum.TypeId);
+            ctx.WriteInt(binEnum.EnumValue);
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/663e78dc/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryUtils.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryUtils.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryUtils.cs
index 1aed03f..1917f01 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryUtils.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryUtils.cs
@@ -23,7 +23,9 @@ namespace Apache.Ignite.Core.Impl.Binary
     using System.Collections.Generic;
     using System.Diagnostics;
     using System.Diagnostics.CodeAnalysis;
+    using System.Globalization;
     using System.IO;
+    using System.Linq;
     using System.Reflection;
     using System.Runtime.InteropServices;
     using System.Text;
@@ -1318,25 +1320,29 @@ namespace Apache.Ignite.Core.Impl.Binary
             }
 
             throw new BinaryObjectException("Only Int32 underlying type is supported for enums: " +
-                enumType.Name);
+                                            enumType.Name);
         }
 
         /// <summary>
-        /// Read enum.
+        /// Gets the enum value by type id and int representation.
         /// </summary>
-        /// <param name="stream">Stream.</param>
-        /// <returns>Enumeration.</returns>
-        public static T ReadEnum<T>(IBinaryStream stream)
+        /// <typeparam name="T">Result type.</typeparam>
+        /// <param name="value">The value.</param>
+        /// <param name="typeId">The type identifier.</param>
+        /// <param name="marsh">The marshaller.</param>
+        /// <returns>value in form of enum, if typeId is known; value in for of int, if typeId is -1.</returns>
+        public static T GetEnumValue<T>(int value, int typeId, Marshaller marsh)
         {
-            if (!typeof(T).IsEnum || Enum.GetUnderlyingType(typeof(T)) == TypInt)
-            {
-                stream.ReadInt();
+            if (typeId == ObjTypeId)
+                return TypeCaster<T>.Cast(value);
 
-                return TypeCaster<T>.Cast(stream.ReadInt());
-            }
+            // All enums are user types
+            var desc = marsh.GetDescriptor(true, typeId);
 
-            throw new BinaryObjectException("Only Int32 underlying type is supported for enums: " +
-                                        typeof (T).Name);
+            if (desc == null || desc.Type == null)
+                throw new BinaryObjectException("Unknown enum type id: " + typeId);
+
+            return (T)Enum.ToObject(desc.Type, value);
         }
 
         /**
@@ -1387,52 +1393,6 @@ namespace Apache.Ignite.Core.Impl.Binary
         }
 
         /**
-         * <summary>Check whether this is predefined type.</summary>
-         * <param name="hdr">Header.</param>
-         * <returns>True is this is one of predefined types with special semantics.</returns>
-         */
-        public static bool IsPredefinedType(byte hdr)
-        {
-            switch (hdr)
-            {
-                case TypeByte:
-                case TypeShort:
-                case TypeInt:
-                case TypeLong:
-                case TypeFloat:
-                case TypeDouble:
-                case TypeChar:
-                case TypeBool:
-                case TypeDecimal:
-                case TypeString:
-                case TypeGuid:
-                case TypeTimestamp:
-                case TypeEnum:
-                case TypeArrayByte:
-                case TypeArrayShort:
-                case TypeArrayInt:
-                case TypeArrayLong:
-                case TypeArrayFloat:
-                case TypeArrayDouble:
-                case TypeArrayChar:
-                case TypeArrayBool:
-                case TypeArrayDecimal:
-                case TypeArrayString:
-                case TypeArrayGuid:
-                case TypeArrayTimestamp:
-                case TypeArrayEnum:
-                case TypeArray:
-                case TypeCollection:
-                case TypeDictionary:
-                case TypeMapEntry:
-                case TypeBinary:
-                    return true;
-                default:
-                    return false;
-            }
-        }
-
-        /**
          * <summary>Convert type name.</summary>
          * <param name="typeName">Type name.</param>
          * <param name="converter">Converter.</param>
@@ -1534,6 +1494,23 @@ namespace Apache.Ignite.Core.Impl.Binary
             return id;
         }
 
+        /// <summary>
+        /// Gets the name of the type.
+        /// </summary>
+        /// <param name="type">The type.</param>
+        /// <returns>
+        /// Simple type name for non-generic types; simple type name with appended generic arguments for generic types.
+        /// </returns>
+        public static string GetTypeName(Type type)
+        {
+            if (!type.IsGenericType)
+                return type.Name;
+
+            var args = type.GetGenericArguments().Select(GetTypeName).Aggregate((x, y) => x + "," + y);
+
+            return string.Format(CultureInfo.InvariantCulture, "{0}[{1}]", type.Name, args);
+        }
+
         /**
          * <summary>Resolve field ID.</summary>
          * <param name="typeId">Type ID.</param>
@@ -1732,7 +1709,8 @@ namespace Apache.Ignite.Core.Impl.Binary
                             IdMapper = CreateInstance<IBinaryIdMapper>(reader),
                             Serializer = CreateInstance<IBinarySerializer>(reader),
                             AffinityKeyFieldName = reader.ReadString(),
-                            KeepDeserialized = reader.ReadObject<bool?>()
+                            KeepDeserialized = reader.ReadObject<bool?>(),
+                            IsEnum = reader.ReadBoolean()
                         });
                     }
                 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/663e78dc/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 c00dad6..189cd50 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryWriter.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryWriter.cs
@@ -20,12 +20,12 @@ namespace Apache.Ignite.Core.Impl.Binary
     using System;
     using System.Collections;
     using System.Collections.Generic;
+    using System.Diagnostics;
     using System.IO;
     using Apache.Ignite.Core.Binary;
     using Apache.Ignite.Core.Impl.Binary.IO;
     using Apache.Ignite.Core.Impl.Binary.Metadata;
     using Apache.Ignite.Core.Impl.Binary.Structure;
-    using Apache.Ignite.Core.Impl.Common;
 
     /// <summary>
     /// Binary writer implementation.
@@ -45,7 +45,7 @@ namespace Apache.Ignite.Core.Impl.Binary
         private BinaryHandleDictionary<object, long> _hnds;
 
         /** Metadatas collected during this write session. */
-        private IDictionary<int, IBinaryType> _metas;
+        private IDictionary<int, BinaryType> _metas;
 
         /** Current type ID. */
         private int _curTypeId;
@@ -803,8 +803,28 @@ namespace Apache.Ignite.Core.Impl.Binary
         /// <param name="val">Enum value.</param>
         public void WriteEnum<T>(T val)
         {
-            _stream.WriteByte(BinaryUtils.TypeEnum);
-            BinaryUtils.WriteEnum(this, val);
+            if (val == null)
+                WriteNullField();
+            else
+            {
+                var desc = _marsh.GetDescriptor(val.GetType());
+
+                if (desc != null)
+                {
+                    var metaHnd = _marsh.GetBinaryTypeHandler(desc);
+
+                    _stream.WriteByte(BinaryUtils.TypeEnum);
+
+                    BinaryUtils.WriteEnum(this, val);
+
+                    SaveMetadata(desc, metaHnd.OnObjectWriteFinished());
+                }
+                else
+                {
+                    // Unregistered enum, write as serializable
+                    Write(new SerializableObjectHolder(val));
+                }
+            }
         }
 
         /// <summary>
@@ -817,10 +837,7 @@ namespace Apache.Ignite.Core.Impl.Binary
         {
             WriteFieldId(fieldName, BinaryUtils.TypeArrayEnum);
 
-            if (val == null)
-                WriteNullField();
-            else
-                WriteEnumArray0(val);
+            WriteEnumArray(val);
         }
 
         /// <summary>
@@ -830,24 +847,26 @@ namespace Apache.Ignite.Core.Impl.Binary
         /// <param name="val">Enum array.</param>
         public void WriteEnumArray<T>(T[] val)
         {
-            if (val == null)
-                WriteNullRawField();
-            else
-                WriteEnumArray0(val);
+            WriteEnumArrayInternal(val, null);
         }
 
         /// <summary>
         /// Writes the enum array.
         /// </summary>
         /// <param name="val">The value.</param>
-        private void WriteEnumArray0<T>(T[] val)
+        /// <param name="elementTypeId">The element type id.</param>
+        public void WriteEnumArrayInternal(Array val, int? elementTypeId)
         {
-            _stream.WriteByte(BinaryUtils.TypeArrayEnum);
+            if (val == null)
+                WriteNullField();
+            else
+            {
+                _stream.WriteByte(BinaryUtils.TypeArrayEnum);
 
-            // typeof(T) can yield wrong results (string[] is object[], for example)
-            var elementType = val.GetType().GetElementType();  
+                var elTypeId = elementTypeId ?? BinaryUtils.GetEnumTypeId(val.GetType().GetElementType(), Marshaller);
 
-            BinaryUtils.WriteArray(val, this, BinaryUtils.GetEnumTypeId(elementType, Marshaller));
+                BinaryUtils.WriteArray(val, this, elTypeId);
+            }
         }
 
         /// <summary>
@@ -1051,12 +1070,20 @@ namespace Apache.Ignite.Core.Impl.Binary
                 return;
             }
 
+            // Handle enums.
+            if (type.IsEnum)
+            {
+                WriteEnum(obj);
+
+                return;
+            }
+
             // Handle special case for builder.
             if (WriteBuilderSpecials(obj))
                 return;
 
             // Suppose that we faced normal object and perform descriptor lookup.
-            IBinaryTypeDescriptor desc = type.IsEnum ? null : _marsh.GetDescriptor(type);
+            IBinaryTypeDescriptor desc = _marsh.GetDescriptor(type);
 
             if (desc != null)
             {
@@ -1356,9 +1383,9 @@ namespace Apache.Ignite.Core.Impl.Binary
         /// Gets collected metadatas.
         /// </summary>
         /// <returns>Collected metadatas (if any).</returns>
-        internal IDictionary<int, IBinaryType> GetBinaryTypes()
+        internal ICollection<BinaryType> GetBinaryTypes()
         {
-            return _metas;
+            return _metas == null ? null : _metas.Values;
         }
 
         /// <summary>
@@ -1399,37 +1426,38 @@ namespace Apache.Ignite.Core.Impl.Binary
         /// <summary>
         /// Saves metadata for this session.
         /// </summary>
-        /// <param name="typeId">Type ID.</param>
-        /// <param name="typeName">Type name.</param>
-        /// <param name="affKeyFieldName">Affinity key field name.</param>
+        /// <param name="desc">The descriptor.</param>
         /// <param name="fields">Fields metadata.</param>
-        internal void SaveMetadata(int typeId, string typeName, string affKeyFieldName, IDictionary<string, int> fields)
+        internal void SaveMetadata(IBinaryTypeDescriptor desc, IDictionary<string, int> fields)
         {
+            Debug.Assert(desc != null);
+
             if (_metas == null)
             {
-                BinaryType meta =
-                    new BinaryType(typeId, typeName, fields, affKeyFieldName);
-
-                _metas = new Dictionary<int, IBinaryType>(1);
-
-                _metas[typeId] = meta;
+                _metas = new Dictionary<int, BinaryType>(1)
+                {
+                    {desc.TypeId, new BinaryType(desc, fields)}
+                };
             }
             else
             {
-                IBinaryType meta;
+                BinaryType meta;
 
-                if (_metas.TryGetValue(typeId, out meta))
+                if (_metas.TryGetValue(desc.TypeId, out meta))
                 {
-                    IDictionary<string, int> existingFields = ((BinaryType)meta).FieldsMap();
-
-                    foreach (KeyValuePair<string, int> field in fields)
+                    if (fields != null)
                     {
-                        if (!existingFields.ContainsKey(field.Key))
-                            existingFields[field.Key] = field.Value;
+                        IDictionary<string, int> existingFields = meta.GetFieldsMap();
+
+                        foreach (KeyValuePair<string, int> field in fields)
+                        {
+                            if (!existingFields.ContainsKey(field.Key))
+                                existingFields[field.Key] = field.Value;
+                        }
                     }
                 }
                 else
-                    _metas[typeId] = new BinaryType(typeId, typeName, fields, affKeyFieldName);
+                    _metas[desc.TypeId] = new BinaryType(desc, fields);
             }
         }
     }