You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2016/06/28 13:07:16 UTC

[29/50] [abbrv] ignite git commit: IGNITE-3328 .NET: Support user-defined AffinityFunction

http://git-wip-us.apache.org/repos/asf/ignite/blob/e1c755c7/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Affinity/AffinityTopologyVersion.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Affinity/AffinityTopologyVersion.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Affinity/AffinityTopologyVersion.cs
new file mode 100644
index 0000000..9bfdfb4
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Affinity/AffinityTopologyVersion.cs
@@ -0,0 +1,138 @@
+/*
+ * 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 Apache.Ignite.Core.Cluster;
+
+    /// <summary>
+    /// Affinity topology version.
+    /// </summary>
+    public struct AffinityTopologyVersion : IEquatable<AffinityTopologyVersion>
+    {
+        /** */
+        private readonly long _version;
+
+        /** */
+        private readonly int _minorVersion;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="AffinityTopologyVersion"/> struct.
+        /// </summary>
+        /// <param name="version">The version.</param>
+        /// <param name="minorVersion">The minor version.</param>
+        public AffinityTopologyVersion(long version, int minorVersion)
+        {
+            _version = version;
+            _minorVersion = minorVersion;
+        }
+
+        /// <summary>
+        /// Gets the major version, same as <see cref="ICluster.TopologyVersion"/>.
+        /// </summary>
+        public long Version
+        {
+            get { return _version; }
+        }
+
+        /// <summary>
+        /// Gets the minor version, which is increased when new caches start.
+        /// </summary>
+        public int MinorVersion
+        {
+            get { return _minorVersion; }
+        }
+
+        /// <summary>
+        /// Indicates whether the current object is equal to another object of the same type.
+        /// </summary>
+        /// <param name="other">An object to compare with this object.</param>
+        /// <returns>
+        /// true if the current object is equal to the <paramref name="other" /> parameter; otherwise, false.
+        /// </returns>
+        public bool Equals(AffinityTopologyVersion other)
+        {
+            return _version == other._version && _minorVersion == other._minorVersion;
+        }
+
+        /// <summary>
+        /// Determines whether the specified <see cref="System.Object" />, is equal to this instance.
+        /// </summary>
+        /// <param name="obj">The <see cref="System.Object" /> to compare with this instance.</param>
+        /// <returns>
+        /// <c>true</c> if the specified <see cref="System.Object" /> is equal to this instance; otherwise, 
+        /// <c>false</c>.
+        /// </returns>
+        public override bool Equals(object obj)
+        {
+            if (ReferenceEquals(null, obj)) return false;
+            return obj is AffinityTopologyVersion && Equals((AffinityTopologyVersion) obj);
+        }
+
+        /// <summary>
+        /// Returns a hash code for this instance.
+        /// </summary>
+        /// <returns>
+        /// A hash code for this instance, suitable for use in hashing algorithms and data structures like a hash table. 
+        /// </returns>
+        public override int GetHashCode()
+        {
+            unchecked
+            {
+                return (_version.GetHashCode()*397) ^ _minorVersion;
+            }
+        }
+
+        /// <summary>
+        /// Implements the operator ==.
+        /// </summary>
+        /// <param name="left">The left.</param>
+        /// <param name="right">The right.</param>
+        /// <returns>
+        /// The result of the operator.
+        /// </returns>
+        public static bool operator ==(AffinityTopologyVersion left, AffinityTopologyVersion right)
+        {
+            return left.Equals(right);
+        }
+
+        /// <summary>
+        /// Implements the operator !=.
+        /// </summary>
+        /// <param name="left">The left.</param>
+        /// <param name="right">The right.</param>
+        /// <returns>
+        /// The result of the operator.
+        /// </returns>
+        public static bool operator !=(AffinityTopologyVersion left, AffinityTopologyVersion right)
+        {
+            return !left.Equals(right);
+        }
+
+        /// <summary>
+        /// Returns a <see cref="string" /> that represents this instance.
+        /// </summary>
+        /// <returns>
+        /// A <see cref="string" /> that represents this instance.
+        /// </returns>
+        public override string ToString()
+        {
+            return string.Format("AffinityTopologyVersion [Version={0}, MinorVersion={1}]", _version, _minorVersion);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/e1c755c7/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
index 1eb00db..b6c190c 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Affinity/IAffinityFunction.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Affinity/IAffinityFunction.cs
@@ -17,19 +17,66 @@
 
 namespace Apache.Ignite.Core.Cache.Affinity
 {
-    using System.Diagnostics.CodeAnalysis;
+    using System;
+    using System.Collections.Generic;
     using Apache.Ignite.Core.Cache.Affinity.Fair;
     using Apache.Ignite.Core.Cache.Affinity.Rendezvous;
+    using Apache.Ignite.Core.Cluster;
 
     /// <summary>
     /// Represents a function that maps cache keys to cluster nodes.
     /// <para />
-    /// Only predefined implementations are supported now: 
+    /// Predefined implementations: 
     /// <see cref="RendezvousAffinityFunction"/>, <see cref="FairAffinityFunction"/>.
     /// </summary>
-    [SuppressMessage("Microsoft.Design", "CA1040:AvoidEmptyInterfaces")]
     public interface IAffinityFunction
     {
-        // No-op.
+        /// <summary>
+        /// Gets the total number of partitions.
+        /// <para />
+        /// All caches should always provide correct partition count which should be the same on all 
+        /// participating nodes. Note that partitions should always be numbered from 0 inclusively 
+        /// to N exclusively without any gaps.
+        /// </summary>
+        int Partitions { get; }
+
+        /// <summary>
+        /// Gets partition number for a given key starting from 0. Partitioned caches
+        /// should make sure that keys are about evenly distributed across all partitions
+        /// from 0 to <see cref="Partitions"/> for best performance.
+        /// <para />
+        /// Note that for fully replicated caches it is possible to segment key sets among different
+        /// grid node groups. In that case each node group should return a unique partition
+        /// number. However, unlike partitioned cache, mappings of keys to nodes in
+        /// replicated caches are constant and a node cannot migrate from one partition
+        /// to another.
+        /// </summary>
+        /// <param name="key">Key to get partition for.</param>
+        /// <returns>Partition number for a given key.</returns>
+        int GetPartition(object key);
+
+        /// <summary>
+        /// Removes node from affinity. This method is called when it is safe to remove 
+        /// disconnected node from affinity mapping.
+        /// </summary>
+        /// <param name="nodeId">The node identifier.</param>
+        void RemoveNode(Guid nodeId);
+
+        /// <summary>
+        /// Gets affinity nodes for a partition. In case of replicated cache, all returned
+        /// nodes are updated in the same manner. In case of partitioned cache, the returned
+        /// list should contain only the primary and back up nodes with primary node being
+        /// always first.
+        /// <pare />
+        /// Note that partitioned affinity must obey the following contract: given that node
+        /// <code>N</code> is primary for some key <code>K</code>, if any other node(s) leave
+        /// grid and no node joins grid, node <code>N</code> will remain primary for key <code>K</code>.
+        /// </summary>
+        /// <param name="context">The affinity function context.</param>
+        /// <returns>
+        /// A collection of partitions, where each partition is a collection of nodes,
+        /// where first node is a primary node, and other nodes are backup nodes.
+        /// </returns>
+        IEnumerable<IEnumerable<IClusterNode>> AssignPartitions(AffinityFunctionContext context);
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/e1c755c7/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 6b37895..54f4753 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs
@@ -28,6 +28,8 @@ namespace Apache.Ignite.Core.Cache.Configuration
     using Apache.Ignite.Core.Binary;
     using Apache.Ignite.Core.Cache;
     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.Eviction;
     using Apache.Ignite.Core.Cache.Store;
     using Apache.Ignite.Core.Common;
@@ -277,7 +279,7 @@ namespace Apache.Ignite.Core.Cache.Configuration
         }
 
         /// <summary>
-        /// Writes this instane to the specified writer.
+        /// Writes this instance to the specified writer.
         /// </summary>
         /// <param name="writer">The writer.</param>
         internal void Write(IBinaryRawWriter writer)
@@ -589,7 +591,7 @@ namespace Apache.Ignite.Core.Cache.Configuration
         public bool ReadFromBackup { get; set; }
 
         /// <summary>
-        /// Gets or sets flag indicating whether copy of of the value stored in cache should be created
+        /// Gets or sets flag indicating whether copy of the value stored in cache should be created
         /// for cache operation implying return value. 
         /// </summary>
         [DefaultValue(DefaultCopyOnRead)]
@@ -665,6 +667,9 @@ namespace Apache.Ignite.Core.Cache.Configuration
 
         /// <summary>
         /// Gets or sets the affinity function to provide mapping from keys to nodes.
+        /// <para />
+        /// Predefined implementations: 
+        /// <see cref="RendezvousAffinityFunction"/>, <see cref="FairAffinityFunction"/>.
         /// </summary>
         public IAffinityFunction AffinityFunction { get; set; }
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/e1c755c7/modules/platforms/dotnet/Apache.Ignite.Core/Events/EventBase.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Events/EventBase.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Events/EventBase.cs
index ed60332..4992266 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Events/EventBase.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Events/EventBase.cs
@@ -195,7 +195,7 @@ namespace Apache.Ignite.Core.Events
         public override string ToString()
         {
             return string.Format(CultureInfo.InvariantCulture, 
-                "CacheEntry [Name={0}, Type={1}, Timestamp={2}, Message={3}]", Name, Type, Timestamp, Message);
+                "{0} [Name={1}, Type={2}, Timestamp={3}, Message={4}]", GetType().Name, Name, Type, Timestamp, Message);
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/e1c755c7/modules/platforms/dotnet/Apache.Ignite.Core/Events/EventReader.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Events/EventReader.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Events/EventReader.cs
index cb1c715..ee1c837 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Events/EventReader.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Events/EventReader.cs
@@ -32,16 +32,14 @@ namespace Apache.Ignite.Core.Events
         /// <param name="reader">Reader.</param>
         /// <returns>Deserialized event.</returns>
         /// <exception cref="System.InvalidCastException">Incompatible event type.</exception>
-        public static T Read<T>(IBinaryReader reader) where T : IEvent
+        public static T Read<T>(IBinaryRawReader reader) where T : IEvent
         {
-            var r = reader.GetRawReader();
-
-            var clsId = r.ReadInt();
+            var clsId = reader.ReadInt();
 
             if (clsId == -1)
                 return default(T);
 
-            return (T) CreateInstance(clsId, r);
+            return (T) CreateInstance(clsId, reader);
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/e1c755c7/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs
index 6bdf1ab..69a2f6a 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs
@@ -37,6 +37,7 @@
     using Apache.Ignite.Core.Impl;
     using Apache.Ignite.Core.Impl.Binary;
     using Apache.Ignite.Core.Impl.Common;
+    using Apache.Ignite.Core.Impl.Handle;
     using Apache.Ignite.Core.Lifecycle;
     using Apache.Ignite.Core.Transactions;
     using BinaryReader = Apache.Ignite.Core.Impl.Binary.BinaryReader;

http://git-wip-us.apache.org/repos/asf/ignite/blob/e1c755c7/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 68fd5bf..24eaa30 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
@@ -33,7 +33,7 @@
                             <xs:element name="typeConfigurations" minOccurs="0">
                                 <xs:complexType>
                                     <xs:sequence>
-                                        <xs:element name="binaryTypeConfiguration">
+                                        <xs:element name="binaryTypeConfiguration" maxOccurs="unbounded">
                                             <xs:complexType>
                                                 <xs:all>
                                                     <xs:element name="nameMapper" minOccurs="0">
@@ -191,7 +191,7 @@
                                         </xs:element>
                                         <xs:element name="affinityFunction" minOccurs="0">
                                             <xs:complexType>
-                                                <xs:attribute name="partitionCount" type="xs:int" />
+                                                <xs:attribute name="partitions" type="xs:int" />
                                                 <xs:attribute name="excludeNeighbors" type="xs:boolean" />
                                                 <xs:attribute name="type" type="xs:string" use="required" />
                                             </xs:complexType>

http://git-wip-us.apache.org/repos/asf/ignite/blob/e1c755c7/modules/platforms/dotnet/Apache.Ignite.Core/Ignition.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Ignition.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Ignition.cs
index 7dc103c..a22c9d0 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Ignition.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Ignition.cs
@@ -272,7 +272,7 @@ namespace Apache.Ignite.Core
         }
 
         /// <summary>
-        /// Preapare configuration.
+        /// Prepare configuration.
         /// </summary>
         /// <param name="reader">Reader.</param>
         /// <param name="outStream">Response stream.</param>
@@ -706,7 +706,7 @@ namespace Apache.Ignite.Core
             /** <inheritdoc /> */
             public void OnLifecycleEvent(LifecycleEventType evt)
             {
-                if (evt == LifecycleEventType.BeforeNodeStop)
+                if (evt == LifecycleEventType.BeforeNodeStop && _ignite != null)
                     ((IgniteProxy) _ignite).Target.BeforeNodeStop();
             }
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/e1c755c7/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbackHandlers.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbackHandlers.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbackHandlers.cs
index fb52033..51d9c74 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbackHandlers.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbackHandlers.cs
@@ -98,5 +98,11 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
 
         internal void* onClientDisconnected;
         internal void* ocClientReconnected;
+
+        internal void* affinityFunctionInit;
+        internal void* affinityFunctionPartition;
+        internal void* affinityFunctionAssignPartitions;
+        internal void* affinityFunctionRemoveNode;
+        internal void* affinityFunctionDestroy;
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/e1c755c7/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs
index f0e881c..89ed838 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs
@@ -21,9 +21,10 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
     using System.Collections.Generic;
     using System.Diagnostics;
     using System.Diagnostics.CodeAnalysis;
+    using System.IO;
     using System.Runtime.InteropServices;
     using System.Threading;
-
+    using Apache.Ignite.Core.Cache.Affinity;
     using Apache.Ignite.Core.Cluster;
     using Apache.Ignite.Core.Common;
     using Apache.Ignite.Core.Impl.Binary;
@@ -165,6 +166,12 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
         private delegate void OnClientDisconnectedDelegate(void* target);
         private delegate void OnClientReconnectedDelegate(void* target, bool clusterRestarted);
 
+        private delegate long AffinityFunctionInitDelegate(void* target, long memPtr);
+        private delegate int AffinityFunctionPartitionDelegate(void* target, long ptr, long memPtr);
+        private delegate void AffinityFunctionAssignPartitionsDelegate(void* target, long ptr, long inMemPtr, long outMemPtr);
+        private delegate void AffinityFunctionRemoveNodeDelegate(void* target, long ptr, long memPtr);
+        private delegate void AffinityFunctionDestroyDelegate(void* target, long ptr);
+
         /// <summary>
         /// constructor.
         /// </summary>
@@ -248,6 +255,12 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
 
                 onClientDisconnected = CreateFunctionPointer((OnClientDisconnectedDelegate)OnClientDisconnected),
                 ocClientReconnected = CreateFunctionPointer((OnClientReconnectedDelegate)OnClientReconnected),
+
+                affinityFunctionInit = CreateFunctionPointer((AffinityFunctionInitDelegate)AffinityFunctionInit),
+                affinityFunctionPartition = CreateFunctionPointer((AffinityFunctionPartitionDelegate)AffinityFunctionPartition),
+                affinityFunctionAssignPartitions = CreateFunctionPointer((AffinityFunctionAssignPartitionsDelegate)AffinityFunctionAssignPartitions),
+                affinityFunctionRemoveNode = CreateFunctionPointer((AffinityFunctionRemoveNodeDelegate)AffinityFunctionRemoveNode),
+                affinityFunctionDestroy = CreateFunctionPointer((AffinityFunctionDestroyDelegate)AffinityFunctionDestroy)
             };
 
             _cbsPtr = Marshal.AllocHGlobal(UU.HandlersSize());
@@ -1089,6 +1102,111 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
 
         #endregion
 
+        #region AffinityFunction
+
+        private long AffinityFunctionInit(void* target, long memPtr)
+        {
+            return SafeCall(() =>
+            {
+                using (var stream = IgniteManager.Memory.Get(memPtr).GetStream())
+                {
+                    var func = _ignite.Marshaller.Unmarshal<IAffinityFunction>(stream);
+
+                    ResourceProcessor.Inject(func, _ignite);
+
+                    return _handleRegistry.Allocate(func);
+                }
+            });
+        }
+
+        private int AffinityFunctionPartition(void* target, long ptr, long memPtr)
+        {
+            return SafeCall(() =>
+            {
+                using (var stream = IgniteManager.Memory.Get(memPtr).GetStream())
+                {
+                    var key = _ignite.Marshaller.Unmarshal<object>(stream);
+
+                    return _handleRegistry.Get<IAffinityFunction>(ptr, true).GetPartition(key);
+                }
+            });
+        }
+
+        private void AffinityFunctionAssignPartitions(void* target, long ptr, long inMemPtr, long outMemPtr)
+        {
+            SafeCall(() =>
+            {
+                using (var inStream = IgniteManager.Memory.Get(inMemPtr).GetStream())
+                {
+                    var ctx = new AffinityFunctionContext(_ignite.Marshaller.StartUnmarshal(inStream));
+                    var func = _handleRegistry.Get<IAffinityFunction>(ptr, true);
+                    var parts = func.AssignPartitions(ctx);
+
+                    if (parts == null)
+                        throw new IgniteException(func.GetType() + ".AssignPartitions() returned invalid result: null");
+
+                    using (var outStream = IgniteManager.Memory.Get(outMemPtr).GetStream())
+                    {
+                        var writer = _ignite.Marshaller.StartMarshal(outStream);
+
+                        var partCnt = 0;
+                        writer.WriteInt(partCnt);  // reserve size
+
+                        foreach (var part in parts)
+                        {
+                            if (part == null)
+                                throw new IgniteException(func.GetType() +
+                                                          ".AssignPartitions() returned invalid partition: null");
+
+                            partCnt++;
+
+                            var nodeCnt = 0;
+                            var cntPos = outStream.Position;
+                            writer.WriteInt(nodeCnt);  // reserve size
+
+                            foreach (var node in part)
+                            {
+                                nodeCnt++;
+                                writer.WriteGuid(node.Id);
+                            }
+
+                            var endPos = outStream.Position;
+                            outStream.Seek(cntPos, SeekOrigin.Begin);
+                            outStream.WriteInt(nodeCnt);
+                            outStream.Seek(endPos, SeekOrigin.Begin);
+                        }
+
+                        outStream.SynchronizeOutput();
+                        outStream.Seek(0, SeekOrigin.Begin);
+                        writer.WriteInt(partCnt);
+                    }
+                }
+            });
+        }
+
+        private void AffinityFunctionRemoveNode(void* target, long ptr, long memPtr)
+        {
+            SafeCall(() =>
+            {
+                using (var stream = IgniteManager.Memory.Get(memPtr).GetStream())
+                {
+                    var nodeId = _ignite.Marshaller.Unmarshal<Guid>(stream);
+
+                    _handleRegistry.Get<IAffinityFunction>(ptr, true).RemoveNode(nodeId);
+                }
+            });
+        }
+
+        private void AffinityFunctionDestroy(void* target, long ptr)
+        {
+            SafeCall(() =>
+            {
+                _handleRegistry.Release(ptr);
+            });
+        }
+
+        #endregion
+
         #region HELPERS
 
         [SuppressMessage("Microsoft.Design", "CA1031:DoNotCatchGeneralExceptionTypes")]

http://git-wip-us.apache.org/repos/asf/ignite/blob/e1c755c7/modules/platforms/dotnet/Apache.Ignite.sln.DotSettings
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.sln.DotSettings b/modules/platforms/dotnet/Apache.Ignite.sln.DotSettings
index 187a909..ac065bc 100644
--- a/modules/platforms/dotnet/Apache.Ignite.sln.DotSettings
+++ b/modules/platforms/dotnet/Apache.Ignite.sln.DotSettings
@@ -1,4 +1,5 @@
 \ufeff<wpf:ResourceDictionary xml:space="preserve" xmlns:x="http://schemas.microsoft.com/winfx/2006/xaml" xmlns:s="clr-namespace:System;assembly=mscorlib" xmlns:ss="urn:shemas-jetbrains-com:settings-storage-xaml" xmlns:wpf="http://schemas.microsoft.com/winfx/2006/xaml/presentation">
-	<s:Boolean x:Key="/Default/CodeStyle/CSharpUsing/AddImportsToDeepestScope/@EntryValue">True</s:Boolean>
-	
-	<s:Boolean x:Key="/Default/CodeStyle/CSharpUsing/QualifiedUsingAtNestedScope/@EntryValue">True</s:Boolean></wpf:ResourceDictionary>
\ No newline at end of file
+	<s:String x:Key="/Default/CodeInspection/CSharpLanguageProject/LanguageLevel/@EntryValue">CSharp50</s:String>
+	<s:Boolean x:Key="/Default/CodeStyle/CSharpUsing/AddImportsToDeepestScope/@EntryValue">True</s:Boolean>	
+	<s:Boolean x:Key="/Default/CodeStyle/CSharpUsing/QualifiedUsingAtNestedScope/@EntryValue">True</s:Boolean>
+</wpf:ResourceDictionary>
\ No newline at end of file