You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by pt...@apache.org on 2016/06/17 11:06:44 UTC

ignite git commit: IGNITE-3320 .NET: Implement CacheConfiguration.AffinityFunction

Repository: ignite
Updated Branches:
  refs/heads/master c96558fad -> 34b5c1b66


IGNITE-3320 .NET: Implement CacheConfiguration.AffinityFunction

This closes #811


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

Branch: refs/heads/master
Commit: 34b5c1b66d4622946336c230d5724f13d3598250
Parents: c96558f
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Fri Jun 17 14:06:11 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Fri Jun 17 14:06:11 2016 +0300

----------------------------------------------------------------------
 .../utils/PlatformConfigurationUtils.java       |  61 +++++++++
 .../Cache/CacheConfigurationTest.cs             | 133 ++++++++++++++++++-
 .../IgniteConfigurationSerializerTest.cs        |  13 ++
 .../IgniteConfigurationTest.cs                  |  12 +-
 .../Apache.Ignite.Core.csproj                   |   4 +
 .../Cache/Affinity/AffinityFunctionBase.cs      | 116 ++++++++++++++++
 .../Cache/Affinity/Fair/FairAffinityFunction.cs |  28 ++++
 .../Cache/Affinity/IAffinityFunction.cs         |  35 +++++
 .../Rendezvous/RendezvousAffinityFunction.cs    |  27 ++++
 .../Cache/Configuration/CacheConfiguration.cs   |   8 ++
 .../Configuration/AtomicConfiguration.cs        |   9 ++
 .../IgniteConfigurationSection.xsd              |   7 +
 .../Transactions/TransactionConfiguration.cs    |  12 ++
 13 files changed, 463 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/34b5c1b6/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 a7b1b17..29b6a70 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,9 @@ 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.cache.affinity.AffinityFunction;
+import org.apache.ignite.cache.affinity.fair.FairAffinityFunction;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
 import org.apache.ignite.cache.eviction.EvictionPolicy;
 import org.apache.ignite.cache.eviction.fifo.FifoEvictionPolicy;
 import org.apache.ignite.cache.eviction.lru.LruEvictionPolicy;
@@ -175,6 +178,7 @@ public class PlatformConfigurationUtils {
             ccfg.setNearConfiguration(readNearConfiguration(in));
 
         ccfg.setEvictionPolicy(readEvictionPolicy(in));
+        ccfg.setAffinity(readAffinityFunction(in));
 
         return ccfg;
     }
@@ -228,6 +232,37 @@ public class PlatformConfigurationUtils {
     }
 
     /**
+     * Reads the eviction policy.
+     *
+     * @param in Stream.
+     * @return Affinity function.
+     */
+    private static AffinityFunction readAffinityFunction(BinaryRawReader in) {
+        byte plcTyp = in.readByte();
+
+        switch (plcTyp) {
+            case 0:
+                break;
+            case 1: {
+                FairAffinityFunction f = new FairAffinityFunction();
+                f.setPartitions(in.readInt());
+                f.setExcludeNeighbors(in.readBoolean());
+                return f;
+            }
+            case 2: {
+                RendezvousAffinityFunction f = new RendezvousAffinityFunction();
+                f.setPartitions(in.readInt());
+                f.setExcludeNeighbors(in.readBoolean());
+                return f;
+            }
+            default:
+                assert false;
+        }
+
+        return null;
+    }
+
+    /**
      * Reads the near config.
      *
      * @param out Stream.
@@ -244,6 +279,31 @@ public class PlatformConfigurationUtils {
     /**
      * Writes the eviction policy.
      * @param out Stream.
+     * @param f Affinity.
+     */
+    private static void writeAffinityFunction(BinaryRawWriter out, AffinityFunction f) {
+        if (f instanceof FairAffinityFunction) {
+            out.writeByte((byte)1);
+
+            FairAffinityFunction f0 = (FairAffinityFunction)f;
+            out.writeInt(f0.getPartitions());
+            out.writeBoolean(f0.isExcludeNeighbors());
+        }
+        else if (f instanceof RendezvousAffinityFunction) {
+            out.writeByte((byte)2);
+
+            RendezvousAffinityFunction f0 = (RendezvousAffinityFunction)f;
+            out.writeInt(f0.getPartitions());
+            out.writeBoolean(f0.isExcludeNeighbors());
+        }
+        else {
+            out.writeByte((byte)0);
+        }
+    }
+
+    /**
+     * Writes the eviction policy.
+     * @param out Stream.
      * @param p Policy.
      */
     private static void writeEvictionPolicy(BinaryRawWriter out, EvictionPolicy p) {
@@ -627,6 +687,7 @@ public class PlatformConfigurationUtils {
             writer.writeBoolean(false);
 
         writeEvictionPolicy(writer, ccfg.getEvictionPolicy());
+        writeAffinityFunction(writer, ccfg.getAffinity());
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/34b5c1b6/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs
index 8a91f88..eb73abe 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs
@@ -21,6 +21,9 @@ namespace Apache.Ignite.Core.Tests.Cache
     using System.Collections.Generic;
     using System.Linq;
     using Apache.Ignite.Core.Binary;
+    using Apache.Ignite.Core.Cache.Affinity;
+    using Apache.Ignite.Core.Cache.Affinity.Fair;
+    using Apache.Ignite.Core.Cache.Affinity.Rendezvous;
     using Apache.Ignite.Core.Cache.Configuration;
     using Apache.Ignite.Core.Cache.Eviction;
     using Apache.Ignite.Core.Cache.Store;
@@ -39,6 +42,9 @@ namespace Apache.Ignite.Core.Tests.Cache
         private const string CacheName = "cacheName";
 
         /** */
+        private const string CacheName2 = "cacheName2";
+
+        /** */
         private static int _factoryProp;
 
 
@@ -53,7 +59,8 @@ namespace Apache.Ignite.Core.Tests.Cache
                 CacheConfiguration = new List<CacheConfiguration>
                 {
                     new CacheConfiguration(),
-                    GetCustomCacheConfiguration()
+                    GetCustomCacheConfiguration(),
+                    GetCustomCacheConfiguration2()
                 },
                 GridName = CacheName,
                 BinaryConfiguration = new BinaryConfiguration(typeof (Entity))
@@ -95,6 +102,12 @@ namespace Apache.Ignite.Core.Tests.Cache
 
             AssertConfigsAreEqual(GetCustomCacheConfiguration(),
                 _ignite.GetConfiguration().CacheConfiguration.Single(c => c.Name == CacheName));
+
+            AssertConfigsAreEqual(GetCustomCacheConfiguration2(),
+                _ignite.GetCache<int, int>(CacheName2).GetConfiguration());
+
+            AssertConfigsAreEqual(GetCustomCacheConfiguration2(),
+                _ignite.GetConfiguration().CacheConfiguration.Single(c => c.Name == CacheName2));
         }
 
         /// <summary>
@@ -240,6 +253,26 @@ namespace Apache.Ignite.Core.Tests.Cache
             AssertConfigsAreEqual(x.QueryEntities, y.QueryEntities);
             AssertConfigsAreEqual(x.NearConfiguration, y.NearConfiguration);
             AssertConfigsAreEqual(x.EvictionPolicy, y.EvictionPolicy);
+            AssertConfigsAreEqual(x.AffinityFunction, y.AffinityFunction);
+        }
+
+        /// <summary>
+        /// Asserts that two configurations have the same properties.
+        /// </summary>
+        private static void AssertConfigsAreEqual(IAffinityFunction x, IAffinityFunction y)
+        {
+            if (x == null)
+            {
+                Assert.IsNull(y);
+                return;
+            }
+
+            var px = (AffinityFunctionBase) x;
+            var py = (AffinityFunctionBase) y;
+
+            Assert.AreEqual(px.GetType(), py.GetType());
+            Assert.AreEqual(px.PartitionCount, py.PartitionCount);
+            Assert.AreEqual(px.ExcludeNeighbors, py.ExcludeNeighbors);
         }
 
         /// <summary>
@@ -516,6 +549,104 @@ namespace Apache.Ignite.Core.Tests.Cache
                     MaxSize = 26,
                     MaxMemorySize = 2501,
                     BatchSize = 33
+                },
+                AffinityFunction = new RendezvousAffinityFunction
+                {
+                    PartitionCount = 513,
+                    ExcludeNeighbors = true
+                }
+            };
+        }
+        /// <summary>
+        /// Gets the custom cache configuration with some variations.
+        /// </summary>
+        private static CacheConfiguration GetCustomCacheConfiguration2(string name = null)
+        {
+            return new CacheConfiguration
+            {
+                Name = name ?? CacheName2,
+                OffHeapMaxMemory = 1,
+                StartSize = 2,
+                MaxConcurrentAsyncOperations = 3,
+                WriteBehindFlushThreadCount = 4,
+                LongQueryWarningTimeout = TimeSpan.FromSeconds(5),
+                LoadPreviousValue = true,
+                EvictSynchronizedKeyBufferSize = 6,
+                CopyOnRead = true,
+                WriteBehindFlushFrequency = TimeSpan.FromSeconds(6),
+                WriteBehindFlushSize = 7,
+                EvictSynchronized = true,
+                AtomicWriteOrderMode = CacheAtomicWriteOrderMode.Clock,
+                AtomicityMode = CacheAtomicityMode.Transactional,
+                Backups = 8,
+                CacheMode = CacheMode.Partitioned,
+                EagerTtl = true,
+                EnableSwap = true,
+                EvictSynchronizedConcurrencyLevel = 9,
+                EvictSynchronizedTimeout = TimeSpan.FromSeconds(10),
+                Invalidate = true,
+                KeepBinaryInStore = true,
+                LockTimeout = TimeSpan.FromSeconds(11),
+                MaxEvictionOverflowRatio = 0.5f,
+                MemoryMode = CacheMemoryMode.OnheapTiered,
+                ReadFromBackup = true,
+                RebalanceBatchSize = 12,
+                RebalanceDelay = TimeSpan.FromSeconds(13),
+                RebalanceMode = CacheRebalanceMode.Sync,
+                RebalanceThrottle = TimeSpan.FromSeconds(15),
+                RebalanceTimeout = TimeSpan.FromSeconds(16),
+                SqlEscapeAll = true,
+                SqlOnheapRowCacheSize = 17,
+                WriteBehindBatchSize = 18,
+                WriteBehindEnabled = false,
+                WriteSynchronizationMode = CacheWriteSynchronizationMode.PrimarySync,
+                CacheStoreFactory = new CacheStoreFactoryTest(),
+                ReadThrough = true,
+                WriteThrough = true,
+                QueryEntities = new[]
+                {
+                    new QueryEntity
+                    {
+                        KeyTypeName = "Integer",
+                        ValueTypeName = "java.lang.String",
+                        Fields = new[]
+                        {
+                            new QueryField("length", typeof(int)), 
+                            new QueryField("name", typeof(string)), 
+                            new QueryField("location", typeof(string)),
+                        },
+                        Aliases = new [] {new QueryAlias("length", "len") },
+                        Indexes = new[]
+                        {
+                            new QueryIndex("name") {Name = "index1" },
+                            new QueryIndex(new QueryIndexField("location", true))
+                            {
+                                Name= "index2",
+                                IndexType = QueryIndexType.Sorted
+                            }
+                        }
+                    }
+                },
+                NearConfiguration = new NearCacheConfiguration
+                {
+                    NearStartSize = 456,
+                    EvictionPolicy = new FifoEvictionPolicy
+                    {
+                        MaxSize = 25,
+                        MaxMemorySize = 2500,
+                        BatchSize = 3
+                    }
+                },
+                EvictionPolicy = new LruEvictionPolicy
+                {
+                    MaxSize = 26,
+                    MaxMemorySize = 2501,
+                    BatchSize = 33
+                },
+                AffinityFunction = new FairAffinityFunction
+                {
+                    PartitionCount = 113,
+                    ExcludeNeighbors = false
                 }
             };
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/34b5c1b6/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 b6ee5cb..3056273 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
@@ -30,6 +30,8 @@ namespace Apache.Ignite.Core.Tests
     using System.Xml;
     using System.Xml.Schema;
     using Apache.Ignite.Core.Binary;
+    using Apache.Ignite.Core.Cache.Affinity.Fair;
+    using Apache.Ignite.Core.Cache.Affinity.Rendezvous;
     using Apache.Ignite.Core.Cache.Configuration;
     using Apache.Ignite.Core.Cache.Eviction;
     using Apache.Ignite.Core.Cache.Store;
@@ -95,6 +97,7 @@ namespace Apache.Ignite.Core.Tests
                                     <nearConfiguration nearStartSize='7'>
                                         <evictionPolicy type='FifoEvictionPolicy' batchSize='10' maxSize='20' maxMemorySize='30' />
                                     </nearConfiguration>
+                                    <affinityFunction type='RendezvousAffinityFunction' partitionCount='99' excludeNeighbors='true' />
                                 </cacheConfiguration>
                                 <cacheConfiguration name='secondCache' />
                             </cacheConfiguration>
@@ -167,6 +170,11 @@ namespace Apache.Ignite.Core.Tests
             Assert.AreEqual(2, plc2.MaxSize);
             Assert.AreEqual(3, plc2.MaxMemorySize);
 
+            var af = cacheCfg.AffinityFunction as RendezvousAffinityFunction;
+            Assert.IsNotNull(af);
+            Assert.AreEqual(99, af.PartitionCount);
+            Assert.IsTrue(af.ExcludeNeighbors);
+
             Assert.AreEqual(new Dictionary<string, object> {{"myNode", "true"}}, cfg.UserAttributes);
 
             var atomicCfg = cfg.AtomicConfiguration;
@@ -436,6 +444,11 @@ namespace Apache.Ignite.Core.Tests
                         EvictionPolicy = new LruEvictionPolicy
                         {
                             BatchSize = 18, MaxMemorySize = 1023, MaxSize = 554
+                        },
+                        AffinityFunction = new FairAffinityFunction
+                        {
+                            ExcludeNeighbors = true,
+                            PartitionCount = 48
                         }
                     }
                 },

http://git-wip-us.apache.org/repos/asf/ignite/blob/34b5c1b6/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 9d11ad3..0c0f62c 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
@@ -22,7 +22,10 @@ namespace Apache.Ignite.Core.Tests
     using System.IO;
     using System.Linq;
     using Apache.Ignite.Core.Binary;
+    using Apache.Ignite.Core.Cache.Affinity.Fair;
+    using Apache.Ignite.Core.Cache.Affinity.Rendezvous;
     using Apache.Ignite.Core.Cache.Configuration;
+    using Apache.Ignite.Core.Cache.Eviction;
     using Apache.Ignite.Core.Common;
     using Apache.Ignite.Core.Communication.Tcp;
     using Apache.Ignite.Core.DataStructures.Configuration;
@@ -68,6 +71,13 @@ namespace Apache.Ignite.Core.Tests
             CheckDefaultValueAttributes(new CacheConfiguration());
             CheckDefaultValueAttributes(new TcpDiscoveryMulticastIpFinder());
             CheckDefaultValueAttributes(new TcpCommunicationSpi());
+            CheckDefaultValueAttributes(new RendezvousAffinityFunction());
+            CheckDefaultValueAttributes(new FairAffinityFunction());
+            CheckDefaultValueAttributes(new NearCacheConfiguration());
+            CheckDefaultValueAttributes(new FifoEvictionPolicy());
+            CheckDefaultValueAttributes(new LruEvictionPolicy());
+            CheckDefaultValueAttributes(new AtomicConfiguration());
+            CheckDefaultValueAttributes(new TransactionConfiguration());
         }
 
         /// <summary>
@@ -369,7 +379,7 @@ namespace Apache.Ignite.Core.Tests
                 var propValue = prop.GetValue(obj, null);
 
                 if (attr != null)
-                    Assert.AreEqual(attr.Value, propValue);
+                    Assert.AreEqual(attr.Value, propValue, string.Format("{0}.{1}", obj.GetType(), prop.Name));
                 else if (prop.PropertyType.IsValueType)
                     Assert.AreEqual(Activator.CreateInstance(prop.PropertyType), propValue);
                 else

http://git-wip-us.apache.org/repos/asf/ignite/blob/34b5c1b6/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 10971d9..4cfd4e5 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
@@ -85,8 +85,12 @@
   <ItemGroup>
     <Compile Include="Binary\BinaryReflectiveSerializer.cs" />
     <Compile Include="Binary\Package-Info.cs" />
+    <Compile Include="Cache\Affinity\AffinityFunctionBase.cs" />
     <Compile Include="Cache\Affinity\AffinityKey.cs" />
     <Compile Include="Cache\Affinity\AffinityKeyMappedAttribute.cs" />
+    <Compile Include="Cache\Affinity\Fair\FairAffinityFunction.cs" />
+    <Compile Include="Cache\Affinity\IAffinityFunction.cs" />
+    <Compile Include="Cache\Affinity\Rendezvous\RendezvousAffinityFunction.cs" />
     <Compile Include="Cache\CacheAtomicUpdateTimeoutException.cs" />
     <Compile Include="Cache\CacheEntryProcessorException.cs" />
     <Compile Include="Cache\CacheException.cs" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/34b5c1b6/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Affinity/AffinityFunctionBase.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Affinity/AffinityFunctionBase.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Affinity/AffinityFunctionBase.cs
new file mode 100644
index 0000000..ea5b21c
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Affinity/AffinityFunctionBase.cs
@@ -0,0 +1,116 @@
+\ufeff/*
+ * 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.Cache.Affinity
+{
+    using System;
+    using System.ComponentModel;
+    using Apache.Ignite.Core.Binary;
+    using Apache.Ignite.Core.Cache.Affinity.Fair;
+    using Apache.Ignite.Core.Cache.Affinity.Rendezvous;
+
+    /// <summary>
+    /// Base class for predefined affinity functions.
+    /// </summary>
+    public abstract class AffinityFunctionBase : IAffinityFunction
+    {
+        /** */
+        private const byte TypeCodeNull = 0;
+
+        /** */
+        private const byte TypeCodeFair = 1;
+
+        /** */
+        private const byte TypeCodeRendezvous = 2;
+
+        /// <summary> The default value for <see cref="PartitionCount"/> property. </summary>
+        public const int DefaultPartitionCount = 1024;
+
+        /// <summary>
+        /// Gets or sets the total number of partitions.
+        /// </summary>
+        [DefaultValue(DefaultPartitionCount)]
+        public int PartitionCount { get; set; }
+
+        /// <summary>
+        /// Gets or sets a value indicating whether to exclude same-host-neighbors from being backups of each other.
+        /// </summary>
+        public bool ExcludeNeighbors { get; set; }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="AffinityFunctionBase"/> class.
+        /// </summary>
+        internal AffinityFunctionBase()
+        {
+            PartitionCount = DefaultPartitionCount;
+        }
+
+        /// <summary>
+        /// Reads the instance.
+        /// </summary>
+        internal static IAffinityFunction Read(IBinaryRawReader reader)
+        {
+            AffinityFunctionBase fun;
+
+            var typeCode = reader.ReadByte();
+            switch (typeCode)
+            {
+                case TypeCodeNull:
+                    return null;
+                case TypeCodeFair:
+                    fun = new FairAffinityFunction();
+                    break;
+                case TypeCodeRendezvous:
+                    fun = new RendezvousAffinityFunction();
+                    break;
+                default:
+                    throw new InvalidOperationException("Invalid AffinityFunction type code: " + typeCode);
+            }
+
+            fun.PartitionCount = reader.ReadInt();
+            fun.ExcludeNeighbors = reader.ReadBoolean();
+
+            return fun;
+        }
+
+        /// <summary>
+        /// Writes the instance.
+        /// </summary>
+        internal static void Write(IBinaryRawWriter writer, IAffinityFunction fun)
+        {
+            if (fun == null)
+            {
+                writer.WriteByte(TypeCodeNull);
+                return;
+            }
+
+            var p = fun as AffinityFunctionBase;
+
+            if (p == null)
+            {
+                throw new NotSupportedException(
+                    string.Format("Unsupported AffinityFunction: {0}. Only predefined affinity function types " +
+                                  "are supported: {1}, {2}", fun.GetType(), typeof(FairAffinityFunction),
+                        typeof(RendezvousAffinityFunction)));
+            }
+
+            writer.WriteByte(p is FairAffinityFunction ? TypeCodeFair : TypeCodeRendezvous);
+            writer.WriteInt(p.PartitionCount);
+            writer.WriteBoolean(p.ExcludeNeighbors);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/34b5c1b6/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Affinity/Fair/FairAffinityFunction.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Affinity/Fair/FairAffinityFunction.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Affinity/Fair/FairAffinityFunction.cs
new file mode 100644
index 0000000..66fb4bc
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Affinity/Fair/FairAffinityFunction.cs
@@ -0,0 +1,28 @@
+/*
+ * 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.Cache.Affinity.Fair
+{
+    /// <summary>
+    /// Fair affinity function which tries to ensure that all nodes get equal number of partitions with 
+    /// minimum amount of reassignments between existing nodes.
+    /// </summary>
+    public class FairAffinityFunction : AffinityFunctionBase
+    {
+        // No-op.
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/34b5c1b6/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Affinity/IAffinityFunction.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Affinity/IAffinityFunction.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Affinity/IAffinityFunction.cs
new file mode 100644
index 0000000..1eb00db
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Affinity/IAffinityFunction.cs
@@ -0,0 +1,35 @@
+\ufeff/*
+ * 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.Cache.Affinity
+{
+    using System.Diagnostics.CodeAnalysis;
+    using Apache.Ignite.Core.Cache.Affinity.Fair;
+    using Apache.Ignite.Core.Cache.Affinity.Rendezvous;
+
+    /// <summary>
+    /// Represents a function that maps cache keys to cluster nodes.
+    /// <para />
+    /// Only predefined implementations are supported now: 
+    /// <see cref="RendezvousAffinityFunction"/>, <see cref="FairAffinityFunction"/>.
+    /// </summary>
+    [SuppressMessage("Microsoft.Design", "CA1040:AvoidEmptyInterfaces")]
+    public interface IAffinityFunction
+    {
+        // No-op.
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/34b5c1b6/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Affinity/Rendezvous/RendezvousAffinityFunction.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Affinity/Rendezvous/RendezvousAffinityFunction.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Affinity/Rendezvous/RendezvousAffinityFunction.cs
new file mode 100644
index 0000000..edc6af0
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Affinity/Rendezvous/RendezvousAffinityFunction.cs
@@ -0,0 +1,27 @@
+/*
+ * 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.Cache.Affinity.Rendezvous
+{
+    /// <summary>
+    /// Affinity function for partitioned cache based on Highest Random Weight algorithm.
+    /// </summary>
+    public class RendezvousAffinityFunction : AffinityFunctionBase
+    {
+        // No-op.
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/34b5c1b6/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs
index 627e244..6b37895 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs
@@ -27,6 +27,7 @@ namespace Apache.Ignite.Core.Cache.Configuration
     using System.Linq;
     using Apache.Ignite.Core.Binary;
     using Apache.Ignite.Core.Cache;
+    using Apache.Ignite.Core.Cache.Affinity;
     using Apache.Ignite.Core.Cache.Eviction;
     using Apache.Ignite.Core.Cache.Store;
     using Apache.Ignite.Core.Common;
@@ -272,6 +273,7 @@ namespace Apache.Ignite.Core.Cache.Configuration
             NearConfiguration = reader.ReadBoolean() ? new NearCacheConfiguration(reader) : null;
 
             EvictionPolicy = EvictionPolicyBase.Read(reader);
+            AffinityFunction = AffinityFunctionBase.Read(reader);
         }
 
         /// <summary>
@@ -344,6 +346,7 @@ namespace Apache.Ignite.Core.Cache.Configuration
                 writer.WriteBoolean(false);
 
             EvictionPolicyBase.Write(writer, EvictionPolicy);
+            AffinityFunctionBase.Write(writer, AffinityFunction);
         }
 
         /// <summary>
@@ -659,5 +662,10 @@ namespace Apache.Ignite.Core.Cache.Configuration
         /// Null value means disabled evictions.
         /// </summary>
         public IEvictionPolicy EvictionPolicy { get; set; }
+
+        /// <summary>
+        /// Gets or sets the affinity function to provide mapping from keys to nodes.
+        /// </summary>
+        public IAffinityFunction AffinityFunction { get; set; }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/34b5c1b6/modules/platforms/dotnet/Apache.Ignite.Core/DataStructures/Configuration/AtomicConfiguration.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/DataStructures/Configuration/AtomicConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/DataStructures/Configuration/AtomicConfiguration.cs
index c6fcbbf..02af6b8 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/DataStructures/Configuration/AtomicConfiguration.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/DataStructures/Configuration/AtomicConfiguration.cs
@@ -54,5 +54,14 @@ namespace Apache.Ignite.Core.DataStructures.Configuration
         /// </summary>
         [DefaultValue(DefaultAtomicSequenceReserveSize)]
         public int AtomicSequenceReserveSize { get; set; }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="AtomicConfiguration" /> class.
+        /// </summary>
+        public AtomicConfiguration()
+        {
+            CacheMode = DefaultCacheMode;
+            AtomicSequenceReserveSize = DefaultAtomicSequenceReserveSize;
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/34b5c1b6/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 6bc2e8f..68fd5bf 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
@@ -189,6 +189,13 @@
                                                 <xs:attribute name="type" type="xs:string" use="required" />
                                             </xs:complexType>
                                         </xs:element>
+                                        <xs:element name="affinityFunction" minOccurs="0">
+                                            <xs:complexType>
+                                                <xs:attribute name="partitionCount" type="xs:int" />
+                                                <xs:attribute name="excludeNeighbors" type="xs:boolean" />
+                                                <xs:attribute name="type" type="xs:string" use="required" />
+                                            </xs:complexType>
+                                        </xs:element>
                                     </xs:all>
                                     <xs:attribute name="name" type="xs:string" />
                                     <xs:attribute name="cacheMode" type="xs:string" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/34b5c1b6/modules/platforms/dotnet/Apache.Ignite.Core/Transactions/TransactionConfiguration.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Transactions/TransactionConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Transactions/TransactionConfiguration.cs
index 8703a79..5fe37f8 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Transactions/TransactionConfiguration.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Transactions/TransactionConfiguration.cs
@@ -72,5 +72,17 @@ namespace Apache.Ignite.Core.Transactions
         /// </summary>
         [DefaultValue(typeof(TimeSpan), "00:00:10")]
         public TimeSpan PessimisticTransactionLogLinger { get; set; }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="TransactionConfiguration" /> class.
+        /// </summary>
+        public TransactionConfiguration()
+        {
+            DefaultTransactionConcurrency = DefaultDefaultTransactionConcurrency;
+            DefaultTransactionIsolation = DefaultDefaultTransactionIsolation;
+            DefaultTimeout = DefaultDefaultTimeout;
+            PessimisticTransactionLogSize = DefaultPessimisticTransactionLogSize;
+            PessimisticTransactionLogLinger = DefaultPessimisticTransactionLogLinger;
+        }
     }
 }