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 2017/05/29 13:48:50 UTC

[1/2] ignite git commit: IGNITE-2492 .NET: Peer assembly loading (enabled for Compute)

Repository: ignite
Updated Branches:
  refs/heads/master 42293fac8 -> 69876116d


http://git-wip-us.apache.org/repos/asf/ignite/blob/69876116/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeActionJob.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeActionJob.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeActionJob.cs
index 55332d5..a1f84da 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeActionJob.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeActionJob.cs
@@ -21,6 +21,7 @@ namespace Apache.Ignite.Core.Impl.Compute.Closure
     using Apache.Ignite.Core.Binary;
     using Apache.Ignite.Core.Compute;
     using Apache.Ignite.Core.Impl.Binary;
+    using Apache.Ignite.Core.Impl.Deployment;
     using Apache.Ignite.Core.Impl.Resource;
 
     /// <summary>
@@ -63,9 +64,9 @@ namespace Apache.Ignite.Core.Impl.Compute.Closure
         /** <inheritDoc /> */
         public void WriteBinary(IBinaryWriter writer)
         {
-            var writer0 = (BinaryWriter)writer.GetRawWriter();
+            var writer0 = (BinaryWriter) writer.GetRawWriter();
 
-            writer0.WithDetach(w => w.WriteObject(_action));
+            writer0.WithDetach(w => w.WriteWithPeerDeployment(_action));
         }
 
         /// <summary>
@@ -74,7 +75,7 @@ namespace Apache.Ignite.Core.Impl.Compute.Closure
         /// <param name="reader">The reader.</param>
         public ComputeActionJob(IBinaryRawReader reader)
         {
-            _action = reader.ReadObject<IComputeAction>();
+            _action = (IComputeAction) reader.ReadObject<object>();
         }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/69876116/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeFuncJob.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeFuncJob.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeFuncJob.cs
index 0cd8df2..8350818 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeFuncJob.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeFuncJob.cs
@@ -20,6 +20,7 @@ namespace Apache.Ignite.Core.Impl.Compute.Closure
     using System;
     using Apache.Ignite.Core.Binary;
     using Apache.Ignite.Core.Impl.Binary;
+    using Apache.Ignite.Core.Impl.Deployment;
     using Apache.Ignite.Core.Impl.Resource;
 
     /// <summary>
@@ -67,8 +68,8 @@ namespace Apache.Ignite.Core.Impl.Compute.Closure
         {
             BinaryWriter writer0 = (BinaryWriter) writer.GetRawWriter();
 
-            writer0.WithDetach(w => w.WriteObject(_clo));
-            writer0.WithDetach(w => w.WriteObject(_arg));
+            writer0.WithDetach(w => w.WriteWithPeerDeployment(_clo));
+            writer0.WithDetach(w => w.WriteWithPeerDeployment(_arg));
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/69876116/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeFunc.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeFunc.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeFunc.cs
index 951e179..4132347 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeFunc.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeFunc.cs
@@ -23,6 +23,7 @@ namespace Apache.Ignite.Core.Impl.Compute
     using Apache.Ignite.Core.Compute;
     using Apache.Ignite.Core.Impl.Binary;
     using Apache.Ignite.Core.Impl.Common;
+    using Apache.Ignite.Core.Impl.Deployment;
     using Apache.Ignite.Core.Impl.Resource;
     using Apache.Ignite.Core.Resource;
 
@@ -76,9 +77,9 @@ namespace Apache.Ignite.Core.Impl.Compute
         /** <inheritDoc /> */
         public void WriteBinary(IBinaryWriter writer)
         {
-            var writer0 = (BinaryWriter)writer.GetRawWriter();
+            var writer0 = (BinaryWriter) writer.GetRawWriter();
 
-            writer0.WithDetach(w => w.WriteObject(_func));
+            writer0.WithDetach(w => w.WriteWithPeerDeployment(_func));
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/69876116/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeImpl.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeImpl.cs
index 66e5339..7a028cd 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeImpl.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeImpl.cs
@@ -35,7 +35,6 @@ namespace Apache.Ignite.Core.Impl.Compute
     using Apache.Ignite.Core.Impl.Common;
     using Apache.Ignite.Core.Impl.Compute.Closure;
     using Apache.Ignite.Core.Impl.Unmanaged;
-    using UU = Apache.Ignite.Core.Impl.Unmanaged.UnmanagedUtils;
 
     /// <summary>
     /// Compute implementation.

http://git-wip-us.apache.org/repos/asf/ignite/blob/69876116/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeJob.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeJob.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeJob.cs
index 4c0b536..56e3708 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeJob.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeJob.cs
@@ -23,6 +23,7 @@ namespace Apache.Ignite.Core.Impl.Compute
     using Apache.Ignite.Core.Compute;
     using Apache.Ignite.Core.Impl.Binary;
     using Apache.Ignite.Core.Impl.Common;
+    using Apache.Ignite.Core.Impl.Deployment;
     using Apache.Ignite.Core.Impl.Resource;
     using Apache.Ignite.Core.Resource;
 
@@ -108,7 +109,7 @@ namespace Apache.Ignite.Core.Impl.Compute
         {
             var writer0 = (BinaryWriter)writer.GetRawWriter();
 
-            writer0.WithDetach(w => w.WriteObject(Job));
+            writer0.WithDetach(w => w.WriteWithPeerDeployment(Job));
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/69876116/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeOutFunc.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeOutFunc.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeOutFunc.cs
index 71934d4..4e5f523 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeOutFunc.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeOutFunc.cs
@@ -24,6 +24,7 @@ namespace Apache.Ignite.Core.Impl.Compute
     using Apache.Ignite.Core.Compute;
     using Apache.Ignite.Core.Impl.Binary;
     using Apache.Ignite.Core.Impl.Common;
+    using Apache.Ignite.Core.Impl.Deployment;
     using Apache.Ignite.Core.Impl.Resource;
     using Apache.Ignite.Core.Resource;
 
@@ -82,7 +83,7 @@ namespace Apache.Ignite.Core.Impl.Compute
         {
             var writer0 = (BinaryWriter)writer.GetRawWriter();
 
-            writer0.WithDetach(w => w.WriteObject(_func));
+            writer0.WithDetach(w => w.WriteWithPeerDeployment(_func));
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/69876116/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Deployment/AssemblyLoader.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Deployment/AssemblyLoader.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Deployment/AssemblyLoader.cs
new file mode 100644
index 0000000..47c94e8
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Deployment/AssemblyLoader.cs
@@ -0,0 +1,105 @@
+/*
+ * 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.Deployment
+{
+    using System;
+    using System.Collections.Generic;
+    using System.Diagnostics;
+    using System.IO;
+    using System.Reflection;
+    using Apache.Ignite.Core.Impl.Common;
+
+    /// <summary>
+    /// Handles assembly loading and serialization.
+    /// </summary>
+    internal static class AssemblyLoader
+    {
+        /// <summary>
+        /// Cache of assemblies that are peer-loaded from byte array.
+        /// Keep these byte arrays to be able to send them further, because Location for such assemblies is empty.
+        /// </summary>
+        private static readonly CopyOnWriteConcurrentDictionary<string, KeyValuePair<Assembly, byte[]>>
+            InMemoryAssemblies
+                = new CopyOnWriteConcurrentDictionary<string, KeyValuePair<Assembly, byte[]>>();
+
+        /// <summary>
+        /// Loads the assembly from bytes outside of any context.
+        /// Resulting assembly can only be retrieved with <see cref="GetAssembly"/> call later.
+        /// It won't be located with <see cref="Type.GetType()"/> call.
+        /// </summary>
+        public static Assembly LoadAssembly(byte[] bytes, string assemblyName)
+        {
+            Debug.Assert(bytes != null);
+            Debug.Assert(!string.IsNullOrWhiteSpace(assemblyName));
+
+            return InMemoryAssemblies.GetOrAdd(assemblyName, _ =>
+            {
+                // Load is better for us than LoadFrom: we want to track loaded assemblies manually.
+                // LoadFrom can cause exceptions when multiple versions of the same assembly exist.
+                var asm = Assembly.Load(bytes);
+
+                Debug.Assert(assemblyName == asm.FullName);
+
+                return new KeyValuePair<Assembly, byte[]>(asm, bytes);
+            }).Key;
+        }
+
+        /// <summary>
+        /// Gets the assembly.
+        /// </summary>
+        public static byte[] GetAssemblyBytes(string assemblyName)
+        {
+            Debug.Assert(!string.IsNullOrWhiteSpace(assemblyName));
+
+            KeyValuePair<Assembly, byte[]> res;
+
+            return InMemoryAssemblies.TryGetValue(assemblyName, out res) ? res.Value : null;
+        }
+
+        /// <summary>
+        /// Gets the assembly.
+        /// </summary>
+        public static Assembly GetAssembly(string assemblyName)
+        {
+            Debug.Assert(!string.IsNullOrWhiteSpace(assemblyName));
+
+            KeyValuePair<Assembly, byte[]> res;
+
+            return InMemoryAssemblies.TryGetValue(assemblyName, out res) ? res.Key : null;
+        }
+
+        /// <summary>
+        /// Gets the assembly bytes.
+        /// </summary>
+        public static byte[] GetAssemblyBytes(Assembly assembly)
+        {
+            Debug.Assert(assembly != null);
+            Debug.Assert(!assembly.IsDynamic);
+
+            KeyValuePair<Assembly, byte[]> pair;
+
+            if (InMemoryAssemblies.TryGetValue(assembly.FullName, out pair))
+                return pair.Value;
+
+            if (string.IsNullOrEmpty(assembly.Location))
+                return null;
+
+            return File.ReadAllBytes(assembly.Location);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/69876116/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Deployment/AssemblyRequest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Deployment/AssemblyRequest.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Deployment/AssemblyRequest.cs
new file mode 100644
index 0000000..ac68893
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Deployment/AssemblyRequest.cs
@@ -0,0 +1,68 @@
+/*
+ * 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.Deployment
+{
+    using System.Diagnostics;
+    using Apache.Ignite.Core.Binary;
+    using Apache.Ignite.Core.Impl.Binary;
+
+    /// <summary>
+    /// Peer assembly request.
+    /// </summary>
+    internal class AssemblyRequest : IBinaryWriteAware
+    {
+        /** */
+        private readonly string _assemblyName;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="AssemblyRequest"/> class.
+        /// </summary>
+        /// <param name="assemblyName">Name of the assembly.</param>
+        public AssemblyRequest(string assemblyName)
+        {
+            Debug.Assert(assemblyName != null);
+
+            _assemblyName = assemblyName;
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="AssemblyRequest"/> class.
+        /// </summary>
+        /// <param name="reader">The reader.</param>
+        public AssemblyRequest(IBinaryRawReader reader)
+        {
+            _assemblyName = reader.ReadString();
+        }
+
+        /** <inheritdoc /> */
+        public void WriteBinary(IBinaryWriter writer)
+        {
+            var raw = writer.GetRawWriter();
+
+            raw.WriteString(_assemblyName);
+        }
+
+        /// <summary>
+        /// Gets the name of the assembly.
+        /// </summary>
+        public string AssemblyName
+        {
+            get { return _assemblyName; }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/69876116/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Deployment/AssemblyRequestResult.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Deployment/AssemblyRequestResult.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Deployment/AssemblyRequestResult.cs
new file mode 100644
index 0000000..f4ccfa1
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Deployment/AssemblyRequestResult.cs
@@ -0,0 +1,80 @@
+/*
+ * 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.Deployment
+{
+    using Apache.Ignite.Core.Binary;
+    using Apache.Ignite.Core.Impl.Binary;
+
+    /// <summary>
+    /// Peer assembly request result.
+    /// </summary>
+    internal class AssemblyRequestResult : IBinaryWriteAware
+    {
+        /** Assembly bytes. */
+        private readonly byte[] _assemblyBytes;
+
+        /** Error message. */
+        private readonly string _message;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="AssemblyRequestResult" /> class.
+        /// </summary>
+        /// <param name="assemblyBytes">The assembly bytes.</param>
+        /// <param name="message">The message.</param>
+        public AssemblyRequestResult(byte[] assemblyBytes, string message)
+        {
+            _assemblyBytes = assemblyBytes;
+            _message = message;
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="AssemblyRequestResult"/> class.
+        /// </summary>
+        /// <param name="reader">The reader.</param>
+        public AssemblyRequestResult(IBinaryRawReader reader)
+        {
+            _assemblyBytes = reader.ReadByteArray();
+            _message = reader.ReadString();
+        }
+
+        /** <inheritdoc /> */
+        public void WriteBinary(IBinaryWriter writer)
+        {
+            var raw = writer.GetRawWriter();
+
+            raw.WriteByteArray(_assemblyBytes);
+            raw.WriteString(_message);
+        }
+
+        /// <summary>
+        /// Gets the assembly bytes.
+        /// </summary>
+        public byte[] AssemblyBytes
+        {
+            get { return _assemblyBytes; }
+        }
+
+        /// <summary>
+        /// Gets the message.
+        /// </summary>
+        public string Message
+        {
+            get { return _message; }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/69876116/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Deployment/GetAssemblyFunc.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Deployment/GetAssemblyFunc.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Deployment/GetAssemblyFunc.cs
new file mode 100644
index 0000000..6d54dbf
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Deployment/GetAssemblyFunc.cs
@@ -0,0 +1,77 @@
+/*
+ * 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.Deployment
+{
+    using System.Diagnostics;
+    using Apache.Ignite.Core.Binary;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Compute;
+    using Apache.Ignite.Core.Impl.Binary;
+    using Apache.Ignite.Core.Impl.Common;
+
+    /// <summary>
+    /// Compute func that returns assembly for a specified name.
+    /// </summary>
+    internal class GetAssemblyFunc : IComputeFunc<AssemblyRequest, AssemblyRequestResult>, IBinaryWriteAware
+    {
+        /** <inheritdoc /> */
+        public AssemblyRequestResult Invoke(AssemblyRequest arg)
+        {
+            if (arg == null)
+            {
+                throw new IgniteException("GetAssemblyFunc does not allow null arguments.");
+            }
+
+            if (arg.AssemblyName == null)
+            {
+                throw new IgniteException("GetAssemblyFunc does not allow null AssemblyName.");
+            }
+
+            Debug.WriteLine("Peer assembly request: " + arg.AssemblyName);
+
+            // Try assemblies in main context.
+            var asm = LoadedAssembliesResolver.Instance.GetAssembly(arg.AssemblyName);
+
+            if (asm != null)
+            {
+                if (asm.IsDynamic)
+                {
+                    return new AssemblyRequestResult(null, 
+                        "Peer assembly loading does not support dynamic assemblies: " + asm);
+                }
+
+                return new AssemblyRequestResult(AssemblyLoader.GetAssemblyBytes(asm), null);
+            }
+
+            var bytes = AssemblyLoader.GetAssemblyBytes(arg.AssemblyName);
+
+            if (bytes != null)
+            {
+                return new AssemblyRequestResult(bytes, null);
+            }
+
+            return null;
+        }
+
+        /** <inheritdoc /> */
+        public void WriteBinary(IBinaryWriter writer)
+        {
+            // No-op.
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/69876116/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Deployment/PeerAssemblyResolver.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Deployment/PeerAssemblyResolver.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Deployment/PeerAssemblyResolver.cs
new file mode 100644
index 0000000..607ca57
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Deployment/PeerAssemblyResolver.cs
@@ -0,0 +1,189 @@
+/*
+ * 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.Deployment
+{
+    using System;
+    using System.Collections.Generic;
+    using System.Diagnostics;
+    using System.Reflection;
+    using Apache.Ignite.Core.Cluster;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Compute;
+    using Apache.Ignite.Core.Deployment;
+    using Apache.Ignite.Core.Impl.Binary;
+    using Apache.Ignite.Core.Impl.Common;
+
+    /// <summary>
+    /// Loads assemblies from other nodes.
+    /// </summary>
+    internal sealed class PeerAssemblyResolver : IDisposable
+    {
+        /** Assembly resolve handler. */
+        private readonly ResolveEventHandler _handler;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="PeerAssemblyResolver"/> class.
+        /// </summary>
+        public PeerAssemblyResolver(Ignite ignite, Guid originNodeId)
+        {
+            Debug.Assert(ignite != null);
+
+            _handler = (sender, args) => GetAssembly(ignite, args.Name, originNodeId);
+
+            AppDomain.CurrentDomain.AssemblyResolve += _handler;
+        }
+
+        /** <inheritdoc /> */
+        public void Dispose()
+        {
+            AppDomain.CurrentDomain.AssemblyResolve -= _handler;
+        }
+
+        /// <summary>
+        /// Gets the assembly from remote nodes.
+        /// </summary>
+        /// <param name="typeName">Assembly-qualified type name.</param>
+        /// <param name="ignite">Ignite.</param>
+        /// <param name="originNodeId">Originating node identifier.</param>
+        /// <returns>
+        /// Resulting type or null.
+        /// </returns>
+        public static Type LoadAssemblyAndGetType(string typeName, Ignite ignite, Guid originNodeId)
+        {
+            Debug.Assert(!string.IsNullOrEmpty(typeName));
+
+            var parsedName = TypeNameParser.Parse(typeName);
+
+            var assemblyName = parsedName.GetAssemblyName();
+
+            Debug.Assert(assemblyName != null);
+
+            var asm = GetAssembly(ignite, assemblyName, originNodeId);
+
+            if (asm == null)
+            {
+                return null;
+            }
+
+            // Assembly.GetType does not work for assembly-qualified names. Full name is required without assembly.
+            return asm.GetType(parsedName.GetFullName(), false);
+        }
+
+        /// <summary>
+        /// Gets the assembly.
+        /// </summary>
+        private static Assembly GetAssembly(Ignite ignite, string assemblyName, Guid originNodeId)
+        {
+            return LoadedAssembliesResolver.Instance.GetAssembly(assemblyName)
+                   ?? AssemblyLoader.GetAssembly(assemblyName)
+                   ?? LoadAssembly(ignite, assemblyName, originNodeId);
+        }
+
+        /// <summary>
+        /// Loads the assembly.
+        /// </summary>
+        private static Assembly LoadAssembly(Ignite ignite, string assemblyName, Guid originNodeId)
+        {
+            var res = RequestAssembly(assemblyName, ignite, originNodeId);
+
+            if (res == null)
+                return null;
+
+            return AssemblyLoader.LoadAssembly(res.AssemblyBytes, assemblyName);
+        }
+
+        /// <summary>
+        /// Gets the assembly from remote nodes.
+        /// </summary>
+        /// <param name="assemblyName">Name of the assembly.</param>
+        /// <param name="ignite">Ignite.</param>
+        /// <param name="originNodeId">The origin node identifier.</param>
+        /// <returns>
+        /// Successful result or null.
+        /// </returns>
+        /// <exception cref="IgniteException"></exception>
+        private static AssemblyRequestResult RequestAssembly(string assemblyName, Ignite ignite, Guid originNodeId)
+        {
+            Debug.Assert(assemblyName != null);
+            Debug.Assert(ignite != null);
+
+            if (ignite.Configuration.PeerAssemblyLoadingMode == PeerAssemblyLoadingMode.Disabled)
+                return null;
+
+            Debug.WriteLine("Requesting assembly from other nodes: " + assemblyName);
+
+            // New nodes are not tracked during the loop, since some of the existing nodes caused this call.
+            var func = new GetAssemblyFunc();
+            var req = new AssemblyRequest(assemblyName);
+
+            foreach (var node in GetDotNetNodes(ignite, originNodeId))
+            {
+                var compute = ignite.GetCluster().ForNodeIds(node).GetCompute();
+                var result = ComputeApplySafe(compute, func, req);
+
+                if (result != null)
+                {
+                    if (result.AssemblyBytes != null)
+                    {
+                        return result;
+                    }
+
+                    if (result.Message != null)
+                    {
+                        throw new IgniteException(result.Message);
+                    }
+                }
+            }
+
+            return null;
+        }
+
+        /// <summary>
+        /// Gets the dot net nodes, origin node comes first.
+        /// </summary>
+        private static IEnumerable<Guid> GetDotNetNodes(IIgnite ignite, Guid originNodeId)
+        {
+            yield return originNodeId;
+
+            foreach (var node in ignite.GetCluster().ForDotNet().ForRemotes().GetNodes())
+            {
+                if (node.Id != originNodeId)
+                {
+                    yield return node.Id;
+                }
+            }
+        }
+
+        /// <summary>
+        /// Performs computation ignoring leaving nodes.
+        /// </summary>
+        private static AssemblyRequestResult ComputeApplySafe(ICompute compute, GetAssemblyFunc func,
+            AssemblyRequest req)
+        {
+            try
+            {
+                return compute.Apply(func, req);
+            }
+            catch (ClusterGroupEmptyException)
+            {
+                // Normal situation: node has left.
+                return null;
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/69876116/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Deployment/PeerLoadingExtensions.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Deployment/PeerLoadingExtensions.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Deployment/PeerLoadingExtensions.cs
new file mode 100644
index 0000000..035a041
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Deployment/PeerLoadingExtensions.cs
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.Core.Impl.Deployment
+{
+    using System;
+    using Apache.Ignite.Core.Deployment;
+    using Apache.Ignite.Core.Impl.Binary;
+
+    /// <summary>
+    /// Reader and Writer extensions for peer deployment.
+    /// </summary>
+    internal static class PeerLoadingExtensions
+    {
+        /** */
+        private static readonly Func<object, PeerLoadingObjectHolder> WrapperFunc =
+            x => new PeerLoadingObjectHolder(x);
+
+        /// <summary>
+        /// Writes the object with peer deployment (when enabled) or normally otherwise.
+        /// </summary>
+        public static void WriteWithPeerDeployment(this BinaryWriter writer, object o)
+        {
+            if (writer.Marshaller.IsPeerAssemblyLoadingEnabled())
+            {
+                try
+                {
+                    writer.WrapperFunc = WrapperFunc;
+                    writer.WriteObject(o);
+                }
+                finally
+                {
+                    writer.WrapperFunc = null;
+                }
+            }
+            else
+            {
+                writer.WriteObject(o);
+            }
+        }
+
+        /// <summary>
+        /// Determines whether peer loading is enabled.
+        /// </summary>
+        private static bool IsPeerAssemblyLoadingEnabled(this Marshaller marshaller)
+        {
+            return marshaller != null && marshaller.Ignite != null &&
+                   marshaller.Ignite.Configuration.PeerAssemblyLoadingMode != PeerAssemblyLoadingMode.Disabled;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/69876116/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Deployment/PeerLoadingObjectHolder.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Deployment/PeerLoadingObjectHolder.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Deployment/PeerLoadingObjectHolder.cs
new file mode 100644
index 0000000..584750a
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Deployment/PeerLoadingObjectHolder.cs
@@ -0,0 +1,90 @@
+/*
+ * 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.Deployment
+{
+    using System;
+    using System.Diagnostics;
+    using Apache.Ignite.Core.Binary;
+    using Apache.Ignite.Core.Impl.Binary;
+
+    /// <summary>
+    /// Holds an object which can have it's assembly automatically loaded on remote nodes.
+    /// 
+    /// Contains assembly-qualified type name.
+    /// Types from assemblies with different versions can coexist and will be differentiated properly.
+    /// </summary>
+    internal class PeerLoadingObjectHolder : IBinaryWriteAware
+    {
+        /** Object. */
+        private readonly object _object;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="PeerLoadingObjectHolder"/> class.
+        /// </summary>
+        /// <param name="o">The object.</param>
+        public PeerLoadingObjectHolder(object o)
+        {
+            Debug.Assert(o != null);
+
+            _object = o;
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="PeerLoadingObjectHolder"/> class.
+        /// </summary>
+        public PeerLoadingObjectHolder(BinaryReader reader)
+        {
+            Debug.Assert(reader != null);
+
+            var originNodeId = reader.ReadGuid().GetValueOrDefault();
+            
+            var typeName = reader.ReadString();
+
+            var ignite = reader.Marshaller.Ignite;
+
+            using (new PeerAssemblyResolver(ignite, originNodeId))  // Resolve transitive dependencies when needed.
+            {
+                // Resolve type from existing assemblies or from remote nodes.
+                var type = Type.GetType(typeName, false)
+                           ?? PeerAssemblyResolver.LoadAssemblyAndGetType(typeName, ignite, originNodeId);
+
+                Debug.Assert(type != null);
+
+                _object = reader.Deserialize<object>(type);
+            }
+        }
+
+        /// <summary>
+        /// Gets the object.
+        /// </summary>
+        public object Object
+        {
+            get { return _object; }
+        }
+
+        /** <inheritdoc /> */
+        public void WriteBinary(IBinaryWriter writer)
+        {
+            var writer0 = (BinaryWriter) writer.GetRawWriter();
+
+            writer0.WriteGuid(writer0.Marshaller.Ignite.GetLocalNode().Id);
+            writer0.WriteString(_object.GetType().AssemblyQualifiedName);
+            writer0.WithDetach(w => w.WriteObject(_object));
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/69876116/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Deployment/PeerLoadingObjectHolderSerializer.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Deployment/PeerLoadingObjectHolderSerializer.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Deployment/PeerLoadingObjectHolderSerializer.cs
new file mode 100644
index 0000000..c9a0a45
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Deployment/PeerLoadingObjectHolderSerializer.cs
@@ -0,0 +1,49 @@
+/*
+ * 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.Deployment
+{
+    using System;
+    using Apache.Ignite.Core.Impl.Binary;
+    using Apache.Ignite.Core.Impl.Common;
+
+    /// <summary>
+    /// Serializer for <see cref="PeerLoadingObjectHolder"/>. Unwraps underlying object automatically.
+    /// </summary>
+    internal class PeerLoadingObjectHolderSerializer : IBinarySerializerInternal
+    {
+        /** <inheritdoc /> */
+        public void WriteBinary<T>(T obj, BinaryWriter writer)
+        {
+            TypeCaster<PeerLoadingObjectHolder>.Cast(obj).WriteBinary(writer);
+        }
+
+        /** <inheritdoc /> */
+        public T ReadBinary<T>(BinaryReader reader, IBinaryTypeDescriptor desc, int pos, Type typeOverride)
+        {
+            var holder = new PeerLoadingObjectHolder(reader);
+
+            return (T) holder.Object;
+        }
+
+        /** <inheritdoc /> */
+        public bool SupportsHandles
+        {
+            get { return false; }
+        }
+    }
+}


[2/2] ignite git commit: IGNITE-2492 .NET: Peer assembly loading (enabled for Compute)

Posted by pt...@apache.org.
IGNITE-2492 .NET: Peer assembly loading (enabled for Compute)

This closes #1937


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

Branch: refs/heads/master
Commit: 69876116dd851d7bfbf0d82e6850654cc57d3d0b
Parents: 42293fa
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Mon May 29 16:48:37 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Mon May 29 16:48:37 2017 +0300

----------------------------------------------------------------------
 .../Apache.Ignite.Core.Tests.csproj             |   9 +
 .../Binary/TypeNameParserTest.cs                |   8 +-
 .../Deployment/GetAddressFunc.cs                |  35 ++++
 .../PeerAssemblyLoadingAllApisTest.cs           | 167 ++++++++++++++++
 .../Deployment/PeerAssemblyLoadingTest.cs       | 191 +++++++++++++++++++
 .../PeerAssemblyLoadingVersioningTest.cs        | 164 ++++++++++++++++
 .../Deployment/ProcessNameFunc.cs               |  50 +++++
 .../Deployment/ProcessNameTask.cs               |  74 +++++++
 .../Deployment/peer_assembly_app.config         |  35 ++++
 .../IgniteConfigurationSerializerTest.cs        |  10 +-
 .../Process/IgniteProcess.cs                    |  12 +-
 .../Apache.Ignite.Core.csproj                   |   9 +
 .../Deployment/PeerAssemblyLoadingMode.cs       |  55 ++++++
 .../Apache.Ignite.Core/IgniteConfiguration.cs   |  16 ++
 .../IgniteConfigurationSection.xsd              |  12 ++
 .../dotnet/Apache.Ignite.Core/Ignition.cs       |  18 +-
 .../Impl/Binary/BinarizableSerializer.cs        |   5 +-
 .../Binary/BinaryEqualityComparerSerializer.cs  |   1 -
 .../Impl/Binary/BinaryProcessor.cs              |   2 +-
 .../Impl/Binary/BinaryReader.cs                 |  66 ++++---
 .../BinaryReflectiveSerializerInternal.cs       |   5 +-
 .../Impl/Binary/BinarySystemTypeSerializer.cs   |   2 +-
 .../Impl/Binary/BinaryWriter.cs                 |  24 +++
 .../Impl/Binary/BinaryWriterExtensions.cs       |   9 +-
 .../Impl/Binary/IBinarySerializerInternal.cs    |  10 +-
 .../Impl/Binary/Marshaller.cs                   |  37 +++-
 .../Impl/Binary/SerializableSerializer.cs       |  14 +-
 .../Impl/Binary/TypeResolver.cs                 |   6 +-
 .../Impl/Binary/UserSerializerProxy.cs          |   5 +-
 .../Common/CopyOnWriteConcurrentDictionary.cs   |   8 -
 .../Impl/Common/LoadedAssembliesResolver.cs     |   8 +-
 .../Impl/Compute/Closure/ComputeActionJob.cs    |   7 +-
 .../Impl/Compute/Closure/ComputeFuncJob.cs      |   5 +-
 .../Impl/Compute/ComputeFunc.cs                 |   5 +-
 .../Impl/Compute/ComputeImpl.cs                 |   1 -
 .../Impl/Compute/ComputeJob.cs                  |   3 +-
 .../Impl/Compute/ComputeOutFunc.cs              |   3 +-
 .../Impl/Deployment/AssemblyLoader.cs           | 105 ++++++++++
 .../Impl/Deployment/AssemblyRequest.cs          |  68 +++++++
 .../Impl/Deployment/AssemblyRequestResult.cs    |  80 ++++++++
 .../Impl/Deployment/GetAssemblyFunc.cs          |  77 ++++++++
 .../Impl/Deployment/PeerAssemblyResolver.cs     | 189 ++++++++++++++++++
 .../Impl/Deployment/PeerLoadingExtensions.cs    |  65 +++++++
 .../Impl/Deployment/PeerLoadingObjectHolder.cs  |  90 +++++++++
 .../PeerLoadingObjectHolderSerializer.cs        |  49 +++++
 45 files changed, 1716 insertions(+), 98 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/69876116/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
index 974f858..6d4f34b 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
@@ -75,6 +75,11 @@
     <Compile Include="Binary\BinaryNameMapperTest.cs" />
     <Compile Include="Binary\BinaryReaderWriterTest.cs" />
     <Compile Include="Binary\BinarySelfTestSimpleName.cs" />
+    <Compile Include="Deployment\GetAddressFunc.cs" />
+    <Compile Include="Deployment\PeerAssemblyLoadingAllApisTest.cs" />
+    <Compile Include="Deployment\PeerAssemblyLoadingVersioningTest.cs" />
+    <Compile Include="Deployment\ProcessNameFunc.cs" />
+    <Compile Include="Deployment\ProcessNameTask.cs" />
     <Compile Include="Binary\IO\BinaryStreamsTest.cs" />
     <Compile Include="Binary\JavaBinaryInteropTest.cs" />
     <Compile Include="Binary\JavaTypeMappingTest.cs" />
@@ -86,6 +91,7 @@
     <Compile Include="Binary\Serializable\PrimitivesTest.cs" />
     <Compile Include="Binary\Serializable\SqlDmlTest.cs" />
     <Compile Include="Binary\TypeNameParserTest.cs" />
+    <Compile Include="Deployment\PeerAssemblyLoadingTest.cs" />
     <Compile Include="Binary\TypeResolverTest.cs" />
     <Compile Include="Cache\AddArgCacheEntryProcessor.cs" />
     <Compile Include="Cache\Affinity\AffinityKeyTest.cs" />
@@ -379,6 +385,9 @@
   <ItemGroup>
     <None Include="Apache.Ignite.Core.Tests.nunit" />
     <None Include="Apache.Ignite.Core.Tests.snk" />
+    <None Include="Deployment\peer_assembly_app.config">
+      <CopyToOutputDirectory>PreserveNewest</CopyToOutputDirectory>
+    </None>
     <None Include="custom_app.config">
       <CopyToOutputDirectory>PreserveNewest</CopyToOutputDirectory>
     </None>

http://git-wip-us.apache.org/repos/asf/ignite/blob/69876116/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/TypeNameParserTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/TypeNameParserTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/TypeNameParserTest.cs
index e566a4b..8718d32 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/TypeNameParserTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/TypeNameParserTest.cs
@@ -43,6 +43,7 @@ namespace Apache.Ignite.Core.Tests.Binary
             Assert.AreEqual("x", res.GetName());
             Assert.AreEqual(0, res.NameStart);
             Assert.AreEqual(0, res.NameEnd);
+            Assert.AreEqual(0, res.FullNameEnd);
             Assert.AreEqual(-1, res.AssemblyStart);
             Assert.AreEqual(-1, res.AssemblyEnd);
             Assert.IsNull(res.Generics);
@@ -52,6 +53,7 @@ namespace Apache.Ignite.Core.Tests.Binary
 
             Assert.AreEqual(7, res.NameStart);
             Assert.AreEqual(9, res.NameEnd);
+            Assert.AreEqual(9, res.FullNameEnd);
             Assert.IsNull(res.Generics);
             Assert.AreEqual(-1, res.AssemblyStart);
 
@@ -60,6 +62,7 @@ namespace Apache.Ignite.Core.Tests.Binary
 
             Assert.AreEqual(7, res.NameStart);
             Assert.AreEqual(9, res.NameEnd);
+            Assert.AreEqual(9, res.FullNameEnd);
             Assert.IsNull(res.Generics);
             Assert.AreEqual(12, res.AssemblyStart);
 
@@ -85,7 +88,7 @@ namespace Apache.Ignite.Core.Tests.Binary
             Assert.AreEqual("List`1", res.GetNameWithNamespace());
             Assert.AreEqual("Int", res.Generics.Single().GetName());
             Assert.AreEqual("Int", res.Generics.Single().GetNameWithNamespace());
-            
+
             // Simple name array.
             res = TypeNameParser.Parse("List`1[[Byte[]]]");
             Assert.AreEqual("List`1", res.GetName());
@@ -224,7 +227,7 @@ namespace Apache.Ignite.Core.Tests.Binary
             CheckType(typeof(int[,,]));
             CheckType(typeof(int[][]));
             CheckType(typeof(int[,,,][,,]));
-            
+
             CheckType(typeof(List<int>[]));
             CheckType(typeof(List<int>[,]));
             CheckType(typeof(List<int>[][]));
@@ -266,6 +269,7 @@ namespace Apache.Ignite.Core.Tests.Binary
             }
 
             Assert.AreEqual(type.FullName.Length + 2, res.AssemblyStart);
+            Assert.AreEqual(type.FullName, res.GetFullName());
         }
 
         private class Nested

http://git-wip-us.apache.org/repos/asf/ignite/blob/69876116/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Deployment/GetAddressFunc.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Deployment/GetAddressFunc.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Deployment/GetAddressFunc.cs
new file mode 100644
index 0000000..9d08873
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Deployment/GetAddressFunc.cs
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.Core.Tests.Deployment
+{
+    extern alias ExamplesDll;
+    using Apache.Ignite.Core.Compute;
+    using Address = ExamplesDll::Apache.Ignite.ExamplesDll.Binary.Address;
+
+    /// <summary>
+    /// Function that returns an instance of a class from another assembly.
+    /// </summary>
+    public class GetAddressFunc : IComputeFunc<int, Address>
+    {
+        /** <inheritdoc /> */
+        public Address Invoke(int arg)
+        {
+            return new Address("addr" + arg, arg);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/69876116/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Deployment/PeerAssemblyLoadingAllApisTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Deployment/PeerAssemblyLoadingAllApisTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Deployment/PeerAssemblyLoadingAllApisTest.cs
new file mode 100644
index 0000000..29abda9
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Deployment/PeerAssemblyLoadingAllApisTest.cs
@@ -0,0 +1,167 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.Core.Tests.Deployment
+{
+    extern alias ExamplesDll;
+    using System.Linq;
+    using Apache.Ignite.Core.Tests.Compute;
+    using Apache.Ignite.Core.Tests.Process;
+    using Apache.Ignite.Log4Net;
+    using NUnit.Framework;
+    using Address = ExamplesDll::Apache.Ignite.ExamplesDll.Binary.Address;
+
+    /// <summary>
+    /// Tests all APIs that support peer assembly loading.
+    /// </summary>
+    [Category(TestUtils.CategoryIntensive)]
+    public class PeerAssemblyLoadingAllApisTest
+    {
+        /// <summary>
+        /// Tests Compute.Call.
+        /// </summary>
+        [Test]
+        public void TestComputeCall([Values(true, false)] bool async)
+        {
+            PeerAssemblyLoadingTest.TestDeployment(remoteCompute =>
+            {
+                Assert.AreEqual("Apache.Ignite", async
+                    ? remoteCompute.CallAsync(new ProcessNameFunc()).Result
+                    : remoteCompute.Call(new ProcessNameFunc()));
+            });
+        }
+
+        /// <summary>
+        /// Tests Compute.Call.
+        /// </summary>
+        [Test]
+        public void TestComputeAffinityCall([Values(true, false)] bool async)
+        {
+            PeerAssemblyLoadingTest.TestDeployment(ignite =>
+            {
+                var cache = ignite.GetOrCreateCache<int, int>("myCache");
+
+                var key = TestUtils.GetPrimaryKey(ignite, cache.Name, ignite.GetCluster().ForRemotes().GetNode());
+
+                var res = async
+                    ? ignite.GetCompute().AffinityCallAsync(cache.Name, key, new ProcessNameFunc()).Result
+                    : ignite.GetCompute().AffinityCall(cache.Name, key, new ProcessNameFunc());
+
+                Assert.AreEqual("Apache.Ignite",
+                    res);
+            });
+        }
+
+        /// <summary>
+        /// Tests Compute.Execute.
+        /// </summary>
+        [Test]
+        public void TestComputeExecute([Values(true, false)] bool async)
+        {
+            PeerAssemblyLoadingTest.TestDeployment(remoteCompute =>
+            {
+                // Argument is from different assembly and should be peer deployed as well.
+                var taskArg = new Address("1", 2);
+
+                Assert.AreEqual("Apache.Ignite_Address [street=1, zip=2]", async
+                    ? remoteCompute.ExecuteAsync(new ProcessNameTask(), taskArg).Result
+                    : remoteCompute.Execute(new ProcessNameTask(), taskArg));
+            });
+        }
+
+        /// <summary>
+        /// Tests Compute.Broadcast(IComputeAction).
+        /// </summary>
+        [Test]
+        public void TestComputeBroadcastAction([Values(true, false)] bool async)
+        {
+            PeerAssemblyLoadingTest.TestDeployment(remoteCompute =>
+            {
+                if (async)
+                {
+                    remoteCompute.BroadcastAsync(new ComputeAction()).Wait();
+                }
+                else
+                {
+                    remoteCompute.Broadcast(new ComputeAction());
+                }
+            });
+        }
+
+        /// <summary>
+        /// Tests Compute.Broadcast(IComputeFunc{T}).
+        /// </summary>
+        [Test]
+        public void TestComputeBroadcastOutFunc([Values(true, false)] bool async)
+        {
+            PeerAssemblyLoadingTest.TestDeployment(remoteCompute =>
+            {
+                var results = async
+                    ? remoteCompute.BroadcastAsync(new ProcessNameFunc()).Result
+                    : remoteCompute.Broadcast(new ProcessNameFunc());
+
+                Assert.AreEqual("Apache.Ignite", results.Single());
+            });
+        }
+
+        /// <summary>
+        /// Tests Compute.Broadcast(IComputeFunc{TArg, TRes}).
+        /// </summary>
+        [Test]
+        public void TestComputeBroadcastFunc([Values(true, false)] bool async)
+        {
+            PeerAssemblyLoadingTest.TestDeployment(remoteCompute =>
+            {
+                // Argument requires additional assembly.
+                var taskArg = new IgniteLog4NetLogger();
+
+                var results = async
+                    ? remoteCompute.BroadcastAsync(new ProcessNameArgFunc(), taskArg).Result
+                    : remoteCompute.Broadcast(new ProcessNameArgFunc(), taskArg);
+
+                Assert.AreEqual("Apache.IgniteApache.Ignite.Log4Net.IgniteLog4NetLogger", results.Single());
+            });
+        }
+
+        /// <summary>
+        /// Tests Compute.Apply.
+        /// </summary>
+        [Test]
+        public void TestComputeApply([Values(true, false)] bool async)
+        {
+            PeerAssemblyLoadingTest.TestDeployment(remoteCompute =>
+            {
+                // Argument is from different assembly and should be peer deployed as well.
+                var taskArg = new Address("1", 2);
+
+                Assert.AreEqual("Apache.IgniteAddress [street=1, zip=2]", async
+                    ? remoteCompute.ApplyAsync(new ProcessNameArgFunc(), taskArg).Result
+                    : remoteCompute.Apply(new ProcessNameArgFunc(), taskArg));
+            });
+        }
+
+        /// <summary>
+        /// Tears down the test.
+        /// </summary>
+        [TearDown]
+        public void TearDown()
+        {
+            Ignition.StopAll(true);
+            IgniteProcess.KillAll();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/69876116/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Deployment/PeerAssemblyLoadingTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Deployment/PeerAssemblyLoadingTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Deployment/PeerAssemblyLoadingTest.cs
new file mode 100644
index 0000000..c74375d
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Deployment/PeerAssemblyLoadingTest.cs
@@ -0,0 +1,191 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.Core.Tests.Deployment
+{
+    extern alias ExamplesDll;
+    using System;
+    using System.IO;
+    using System.Threading;
+    using Apache.Ignite.Core.Cluster;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Compute;
+    using Apache.Ignite.Core.Deployment;
+    using Apache.Ignite.Core.Impl;
+    using Apache.Ignite.Core.Impl.Common;
+    using Apache.Ignite.Core.Tests.Process;
+    using Apache.Ignite.Log4Net;
+    using NUnit.Framework;
+    using Address = ExamplesDll::Apache.Ignite.ExamplesDll.Binary.Address;
+
+    /// <summary>
+    /// Tests peer assembly loading feature:
+    /// when user-defined type is not found within loaded assemblies on remote node,
+    /// corresponding assembly is sent and loaded automatically.
+    /// </summary>
+    public class PeerAssemblyLoadingTest
+    {
+        /// <summary>
+        /// Tests that peer loading does not happen when not enabled in config, and error message is informative.
+        /// </summary>
+        [Test]
+        public void TestDisabledPeerLoading()
+        {
+            TestDeployment(remoteCompute =>
+            {
+                var ex = Assert.Throws<IgniteException>(() => remoteCompute.Call(new ProcessNameFunc()));
+
+                Assert.AreEqual("Compute job has failed on remote node, examine InnerException for details.", 
+                    ex.Message);
+
+                Assert.IsNotNull(ex.InnerException);
+                Assert.AreEqual("Failed to deserialize the job [errType=BinaryObjectException, errMsg=No matching " +
+                                "type found for object", ex.InnerException.Message.Substring(0, 102));
+            }, false);
+        }
+
+        /// <summary>
+        /// Tests single assembly deployment (basic test).
+        /// </summary>
+        [Test]
+        public void TestSingleAssembly()
+        {
+            TestDeployment(remoteCompute =>
+            {
+                Assert.AreEqual("Apache.Ignite", remoteCompute.Call(new ProcessNameFunc()));
+            });
+        }
+
+        /// <summary>
+        /// Tests that a type which requires multiple assemblies can be peer deployed.
+        /// </summary>
+        [Test]
+        public void TestMultipleAssemblies()
+        {
+            TestDeployment(remoteCompute =>
+            {
+                // GetAddressFunc requires Tests and Examples assemblies.
+                var result = remoteCompute.Apply(new GetAddressFunc(), 3);
+
+                Assert.IsNotNull(result);
+
+                Assert.AreEqual(3, result.Zip);
+                Assert.AreEqual("addr3", result.Street);
+            });
+        }
+
+        /// <summary>
+        /// Tests that a type which requires multiple assemblies can be peer deployed.
+        /// </summary>
+        [Test]
+        public void TestMultipleAssembliesIndirectDependency()
+        {
+            TestDeployment(remoteCompute =>
+            {
+                // Arg is object, but value is from Examples assembly.
+                Assert.AreEqual("Apache.IgniteAddress [street=Central, zip=2]", remoteCompute.Call(
+                    new ProcessNameFunc {Arg = new Address("Central", 2)}));
+            });
+        }
+
+        /// <summary>
+        /// Tests that a type which requires multiple assemblies can be peer deployed.
+        /// </summary>
+        [Test]
+        public void TestMultipleAssembliesIndirectDependencyMultiLevel()
+        {
+            TestDeployment(remoteCompute =>
+            {
+                // Arg is object, value is from Apache.Ignite.Log4Net, and it further depends on log4net.
+                Assert.AreEqual("Apache.IgniteApache.Ignite.Log4Net.IgniteLog4NetLogger", remoteCompute.Call(
+                    new ProcessNameFunc {Arg = new IgniteLog4NetLogger()}));
+            });
+        }
+
+        /// <summary>
+        /// Tests the peer deployment.
+        /// </summary>
+        public static void TestDeployment(Action<ICompute> test, bool enablePeerDeployment = true)
+        {
+            TestDeployment((IClusterGroup remoteCluster) => test(remoteCluster.GetCompute()), enablePeerDeployment);
+        }
+
+        /// <summary>
+        /// Tests the peer deployment.
+        /// </summary>
+        private static void TestDeployment(Action<IClusterGroup> test, bool enablePeerDeployment = true)
+        {
+            TestDeployment(ignite => test(ignite.GetCluster().ForRemotes()), enablePeerDeployment);
+        }
+
+        /// <summary>
+        /// Tests the peer deployment.
+        /// </summary>
+        public static void TestDeployment(Action<IIgnite> test, bool enablePeerDeployment = true)
+        {
+            // Copy Apache.Ignite.exe and Apache.Ignite.Core.dll 
+            // to a separate folder so that it does not locate our assembly automatically.
+            var folder = IgniteUtils.GetTempDirectoryName();
+            foreach (var asm in new[] {typeof(IgniteRunner).Assembly, typeof(Ignition).Assembly})
+            {
+                Assert.IsNotNull(asm.Location);
+                File.Copy(asm.Location, Path.Combine(folder, Path.GetFileName(asm.Location)));
+            }
+
+            var exePath = Path.Combine(folder, "Apache.Ignite.exe");
+
+            // Start separate Ignite process without loading current dll.
+            // ReSharper disable once AssignNullToNotNullAttribute
+            var config = Path.Combine(Path.GetDirectoryName(typeof(PeerAssemblyLoadingTest).Assembly.Location),
+                "Deployment\\peer_assembly_app.config");
+
+            var proc = IgniteProcess.Start(exePath, IgniteHome.Resolve(null), null,
+                "-ConfigFileName=" + config, "-ConfigSectionName=igniteConfiguration");
+
+            Thread.Sleep(300);
+            Assert.IsFalse(proc.HasExited);
+
+            // Start Ignite and execute computation on remote node.
+            var cfg = new IgniteConfiguration(TestUtils.GetTestConfiguration())
+            {
+                PeerAssemblyLoadingMode = enablePeerDeployment
+                    ? PeerAssemblyLoadingMode.CurrentAppDomain
+                    : PeerAssemblyLoadingMode.Disabled
+            };
+
+            using (var ignite = Ignition.Start(cfg))
+            {
+                Assert.IsTrue(ignite.WaitTopology(2));
+
+                for (var i = 0; i < 10; i++)
+                {
+                    test(ignite);
+                }
+            }
+        }
+
+        /// <summary>
+        /// Tears down the test.
+        /// </summary>
+        [TearDown]
+        public void TearDown()
+        {
+            Ignition.StopAll(true);
+            IgniteProcess.KillAll();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/69876116/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Deployment/PeerAssemblyLoadingVersioningTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Deployment/PeerAssemblyLoadingVersioningTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Deployment/PeerAssemblyLoadingVersioningTest.cs
new file mode 100644
index 0000000..a6fe76e
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Deployment/PeerAssemblyLoadingVersioningTest.cs
@@ -0,0 +1,164 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.Core.Tests.Deployment
+{
+    using System;
+    using System.CodeDom.Compiler;
+    using System.Diagnostics;
+    using System.IO;
+    using Apache.Ignite.Core.Deployment;
+    using Apache.Ignite.Core.Discovery.Tcp;
+    using Apache.Ignite.Core.Discovery.Tcp.Static;
+    using Apache.Ignite.Core.Impl;
+    using Apache.Ignite.Core.Tests.Process;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests assembly versioning: multiple assemblies with same name but different version should be supported.
+    /// </summary>
+    public class PeerAssemblyLoadingVersioningTest
+    {
+        private static readonly string TempDir = IgniteUtils.GetTempDirectoryName();
+
+        /// <summary>
+        /// Sets up the test.
+        /// </summary>
+        [SetUp]
+        public void SetUp()
+        {
+            // Copy referenced assemblies.
+            foreach (var type in new[] { typeof(Ignition), GetType() })
+            {
+                var loc = type.Assembly.Location;
+                Assert.IsNotNull(loc);
+                File.Copy(loc, Path.Combine(TempDir, type.Assembly.GetName().Name + ".dll"));
+            }
+        }
+
+        /// <summary>
+        /// Tears down the test.
+        /// </summary>
+        [TearDown]
+        public void TearDown()
+        {
+            Directory.Delete(TempDir, true);
+        }
+
+        /// <summary>
+        /// Tests that multiple versions of same assembly can be used on remote nodes.
+        /// </summary>
+        [Test]
+        public void TestMultipleVersionsOfSameAssembly()
+        {
+            using (Ignition.Start(new IgniteConfiguration(TestUtils.GetTestConfiguration())
+            {
+                PeerAssemblyLoadingMode = PeerAssemblyLoadingMode.CurrentAppDomain,
+                IgniteInstanceName = "peerDeployTest",
+                DiscoverySpi = new TcpDiscoverySpi
+                {
+                    IpFinder = new TcpDiscoveryStaticIpFinder {Endpoints = new[] {"127.0.0.1:47500..47502"}},
+                    SocketTimeout = TimeSpan.FromSeconds(0.3)
+                }
+            }))
+            {
+                RunClientProcess(CompileClientNode(Path.Combine(TempDir, "PeerTest.exe"), "1.0.1.0"));
+                RunClientProcess(CompileClientNode(Path.Combine(TempDir, "PeerTest.exe"), "1.0.2.0"));
+            }
+        }
+
+        /// <summary>
+        /// Runs the client process.
+        /// </summary>
+        private static void RunClientProcess(string exePath)
+        {
+            var procStart = new ProcessStartInfo
+            {
+                FileName = exePath,
+                CreateNoWindow = true,
+                UseShellExecute = false,
+                RedirectStandardOutput = true,
+                RedirectStandardError = true,
+            };
+
+            var proc = Process.Start(procStart);
+            Assert.IsNotNull(proc);
+
+            IgniteProcess.AttachProcessConsoleReader(proc);
+
+            Assert.IsTrue(proc.WaitForExit(30000));
+            Assert.AreEqual(0, proc.ExitCode);
+
+            File.Delete(exePath);
+        }
+
+        /// <summary>
+        /// Compiles the client node.
+        /// </summary>
+        private string CompileClientNode(string exePath, string version)
+        {
+            var parameters = new CompilerParameters
+            {
+                GenerateExecutable = true,
+                OutputAssembly = exePath,
+                ReferencedAssemblies =
+                {
+                    typeof(Ignition).Assembly.Location,
+                    GetType().Assembly.Location,
+                    "System.dll"
+                }
+            };
+
+            var src = @"
+using System;
+using System.Reflection;
+
+using Apache.Ignite.Core;
+using Apache.Ignite.Core.Deployment;
+using Apache.Ignite.Core.Compute;
+using Apache.Ignite.Core.Tests;
+using Apache.Ignite.Core.Discovery;
+using Apache.Ignite.Core.Discovery.Tcp;
+using Apache.Ignite.Core.Discovery.Tcp.Static;
+
+[assembly: AssemblyVersion(""" + version + @""")]
+
+class Program
+{
+    static void Main(string[] args)
+    {
+        using (var ignite = Ignition.Start(new IgniteConfiguration(TestUtils.GetTestConfiguration(false)) {ClientMode = true, PeerAssemblyLoadingMode = PeerAssemblyLoadingMode.CurrentAppDomain,
+                DiscoverySpi = new TcpDiscoverySpi { IpFinder = new TcpDiscoveryStaticIpFinder { Endpoints = new[] { ""127.0.0.1:47500..47502"" } }, SocketTimeout = TimeSpan.FromSeconds(0.3) }
+}))
+        {
+            var res = ignite.GetCompute().Call(new GridNameFunc());
+            if (res != ""peerDeployTest_" + version + @""") throw new Exception(""fail: "" + res);
+        }
+    }
+}
+
+public class GridNameFunc : IComputeFunc<string> { public string Invoke() { return Ignition.GetIgnite().Name + ""_"" + GetType().Assembly.GetName().Version; } }
+";
+
+            var results = CodeDomProvider.CreateProvider("CSharp").CompileAssemblyFromSource(parameters, src);
+
+            Assert.IsEmpty(results.Errors);
+
+            return exePath;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/69876116/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Deployment/ProcessNameFunc.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Deployment/ProcessNameFunc.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Deployment/ProcessNameFunc.cs
new file mode 100644
index 0000000..846e11a
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Deployment/ProcessNameFunc.cs
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.Core.Tests.Deployment
+{
+    using Apache.Ignite.Core.Compute;
+
+    /// <summary>
+    /// Function that returns parent process name.
+    /// </summary>
+    public class ProcessNameFunc : IComputeFunc<string>
+    {
+        /// <summary>
+        /// Gets or sets the argument.
+        /// </summary>
+        public object Arg { get; set; }
+
+        /** <inheritdoc /> */
+        public string Invoke()
+        {
+            return System.Diagnostics.Process.GetCurrentProcess().ProcessName + Arg;
+        }
+    }
+
+    /// <summary>
+    /// Function that returns parent process name.
+    /// </summary>
+    public class ProcessNameArgFunc : IComputeFunc<object, string>
+    {
+        /** <inheritdoc /> */
+        public string Invoke(object arg)
+        {
+            return System.Diagnostics.Process.GetCurrentProcess().ProcessName + arg;
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/69876116/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Deployment/ProcessNameTask.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Deployment/ProcessNameTask.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Deployment/ProcessNameTask.cs
new file mode 100644
index 0000000..528ad6f
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Deployment/ProcessNameTask.cs
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.Core.Tests.Deployment
+{
+    using System.Collections.Generic;
+    using System.Linq;
+    using Apache.Ignite.Core.Cluster;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Compute;
+
+    /// <summary>
+    /// Task that returns process name.
+    /// </summary>
+    public class ProcessNameTask : IComputeTask<object, string, string>
+    {
+        /** <inheritdoc /> */
+        public IDictionary<IComputeJob<string>, IClusterNode> Map(IList<IClusterNode> subgrid, object arg)
+        {
+            return subgrid.ToDictionary(x => (IComputeJob<string>) new ProcessNameJob {Arg = arg}, x => x);
+        }
+
+        /** <inheritdoc /> */
+        public ComputeJobResultPolicy OnResult(IComputeJobResult<string> res, IList<IComputeJobResult<string>> rcvd)
+        {
+            return ComputeJobResultPolicy.Reduce;
+        }
+
+        /** <inheritdoc /> */
+        public string Reduce(IList<IComputeJobResult<string>> results)
+        {
+            var ex = results.Select(x => x.Exception).FirstOrDefault(x => x != null);
+
+            if (ex != null)
+                throw new IgniteException("Task failed", ex);
+
+            return results.Select(x => x.Data).FirstOrDefault();
+        }
+
+        /// <summary>
+        /// Job.
+        /// </summary>
+        private class ProcessNameJob : IComputeJob<string>
+        {
+            public object Arg { get; set; }
+
+            /** <inheritdoc /> */
+            public string Execute()
+            {
+                return System.Diagnostics.Process.GetCurrentProcess().ProcessName + "_" + Arg;
+            }
+
+            /** <inheritdoc /> */
+            public void Cancel()
+            {
+                // No-op.
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/69876116/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Deployment/peer_assembly_app.config
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Deployment/peer_assembly_app.config b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Deployment/peer_assembly_app.config
new file mode 100644
index 0000000..4cb1d6c
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Deployment/peer_assembly_app.config
@@ -0,0 +1,35 @@
+<?xml version="1.0" encoding="utf-8"?>
+
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+<configuration>
+    <configSections>
+        <section name="igniteConfiguration" type="Apache.Ignite.Core.IgniteConfigurationSection, Apache.Ignite.Core" />
+    </configSections>
+
+    <igniteConfiguration xmlns="http://ignite.apache.org/schema/dotnet/IgniteConfigurationSection"
+                         localhost="127.0.0.1" peerAssemblyLoadingMode="CurrentAppDomain">
+        <discoverySpi type="TcpDiscoverySpi">
+            <ipFinder type="TcpDiscoveryStaticIpFinder">
+                <endpoints>
+                    <string>127.0.0.1:47500</string>
+                </endpoints>
+            </ipFinder>
+        </discoverySpi>
+    </igniteConfiguration>
+</configuration>

http://git-wip-us.apache.org/repos/asf/ignite/blob/69876116/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 5bbf722..a90eae5 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
@@ -41,6 +41,7 @@ namespace Apache.Ignite.Core.Tests
     using Apache.Ignite.Core.Common;
     using Apache.Ignite.Core.Communication.Tcp;
     using Apache.Ignite.Core.DataStructures.Configuration;
+    using Apache.Ignite.Core.Deployment;
     using Apache.Ignite.Core.Discovery.Tcp;
     using Apache.Ignite.Core.Discovery.Tcp.Multicast;
     using Apache.Ignite.Core.Events;
@@ -65,7 +66,7 @@ namespace Apache.Ignite.Core.Tests
         [Test]
         public void TestPredefinedXml()
         {
-            var xml = @"<igniteConfig workDirectory='c:' JvmMaxMemoryMb='1024' MetricsLogFrequency='0:0:10' isDaemon='true' isLateAffinityAssignment='false' springConfigUrl='c:\myconfig.xml' autoGenerateIgniteInstanceName='true'>
+            var xml = @"<igniteConfig workDirectory='c:' JvmMaxMemoryMb='1024' MetricsLogFrequency='0:0:10' isDaemon='true' isLateAffinityAssignment='false' springConfigUrl='c:\myconfig.xml' autoGenerateIgniteInstanceName='true' peerAssemblyLoadingMode='CurrentAppDomain'>
                             <localhost>127.1.1.1</localhost>
                             <binaryConfiguration compactFooter='false' keepDeserialized='true'>
                                 <nameMapper type='Apache.Ignite.Core.Tests.IgniteConfigurationSerializerTest+NameMapper' bar='testBar' />
@@ -92,7 +93,7 @@ namespace Apache.Ignite.Core.Tests
                             <cacheConfiguration>
                                 <cacheConfiguration cacheMode='Replicated' readThrough='true' writeThrough='true' enableStatistics='true' writeBehindCoalescing='false' partitionLossPolicy='ReadWriteAll'>
                                     <queryEntities>    
-                                        <queryEntity keyType='System.Int32' valueType='System.String' tableName='myTable'>    
+                                        <queryEntity keyType='System.Int32' valueType='System.String' tableName='myTable'>
                                             <fields>
                                                 <queryField name='length' fieldType='System.Int32' isKeyField='true' />
                                             </fields>
@@ -273,6 +274,8 @@ namespace Apache.Ignite.Core.Tests
             Assert.AreEqual(89, memPlc.InitialSize);
             Assert.AreEqual(98, memPlc.MaxSize);
             Assert.IsTrue(memPlc.MetricsEnabled);
+
+            Assert.AreEqual(PeerAssemblyLoadingMode.CurrentAppDomain, cfg.PeerAssemblyLoadingMode);
         }
 
         /// <summary>
@@ -831,7 +834,8 @@ namespace Apache.Ignite.Core.Tests
                             MetricsEnabled = true
                         }
                     }
-                }
+                },
+                PeerAssemblyLoadingMode = PeerAssemblyLoadingMode.CurrentAppDomain
             };
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/69876116/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Process/IgniteProcess.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Process/IgniteProcess.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Process/IgniteProcess.cs
index 5aec1ac..ee62354 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Process/IgniteProcess.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Process/IgniteProcess.cs
@@ -197,12 +197,20 @@ namespace Apache.Ignite.Core.Tests.Process
             Debug.Assert(proc != null);
 
             // 3. Attach output readers to avoid hangs.
+            AttachProcessConsoleReader(proc, outReader);
+
+            return proc;
+        }
+
+        /// <summary>
+        /// Attaches the process console reader.
+        /// </summary>
+        public static void AttachProcessConsoleReader(Process proc, IIgniteProcessOutputReader outReader = null)
+        {
             outReader = outReader ?? DfltOutReader;
 
             Attach(proc, proc.StandardOutput, outReader, false);
             Attach(proc, proc.StandardError, outReader, true);
-
-            return proc;
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/69876116/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 9ce9dd2..b37b685 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
@@ -98,6 +98,7 @@
     <Compile Include="Cache\Configuration\PartitionLossPolicy.cs" />
     <Compile Include="Cache\IMemoryMetrics.cs" />
     <Compile Include="Common\ExceptionFactory.cs" />
+    <Compile Include="Deployment\PeerAssemblyLoadingMode.cs" />
     <Compile Include="Events\IEventStorageSpi.cs" />
     <Compile Include="Events\MemoryEventStorageSpi.cs" />
     <Compile Include="Events\NoopEventStorageSpi.cs" />
@@ -111,6 +112,14 @@
     <Compile Include="Impl\Binary\BinaryArrayEqualityComparer.cs" />
     <Compile Include="Impl\Binary\BinaryProcessor.cs" />
     <Compile Include="Impl\Binary\BinaryReflectiveSerializerInternal.cs" />
+    <Compile Include="Impl\Deployment\AssemblyLoader.cs" />
+    <Compile Include="Impl\Deployment\AssemblyRequest.cs" />
+    <Compile Include="Impl\Deployment\AssemblyRequestResult.cs" />
+    <Compile Include="Impl\Deployment\GetAssemblyFunc.cs" />
+    <Compile Include="Impl\Deployment\PeerAssemblyResolver.cs" />
+    <Compile Include="Impl\Deployment\PeerLoadingExtensions.cs" />
+    <Compile Include="Impl\Deployment\PeerLoadingObjectHolder.cs" />
+    <Compile Include="Impl\Deployment\PeerLoadingObjectHolderSerializer.cs" />
     <Compile Include="Impl\Binary\IBinarySerializerInternal.cs" />
     <Compile Include="Binary\Package-Info.cs" />
     <Compile Include="Cache\Affinity\AffinityKey.cs" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/69876116/modules/platforms/dotnet/Apache.Ignite.Core/Deployment/PeerAssemblyLoadingMode.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Deployment/PeerAssemblyLoadingMode.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Deployment/PeerAssemblyLoadingMode.cs
new file mode 100644
index 0000000..7217e2a
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Deployment/PeerAssemblyLoadingMode.cs
@@ -0,0 +1,55 @@
+/*
+ * 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.Deployment
+{
+    using System;
+    using Apache.Ignite.Core.Compute;
+
+    /// <summary>
+    /// Peer assembly loading mode.
+    /// See <see cref="IgniteConfiguration.PeerAssemblyLoadingMode"/>.
+    /// </summary>
+    public enum PeerAssemblyLoadingMode
+    {
+        /// <summary>
+        /// Disabled peer assembly loading. Default mode.
+        /// </summary>
+        Disabled,
+
+        /// <summary>
+        /// Automatically load assemblies from remote nodes into the current <see cref="AppDomain"/>.
+        /// <para />
+        /// .NET does not allow assembly unloading, which means that all peer-loaded assemblies will
+        /// live as long as the current AppDomain lives. This may cause increased memory usage.
+        /// <para />
+        /// Assemblies are distinguished using their fully qualified name. Multiple versions of the same assembly can
+        /// be loaded and the correct version will be used (according to Type.AssemblyQualifiedName).
+        /// So in case when a new version of some type needs to be executed on remote nodes,
+        /// corresponding assembly version should be bumped up. If assembly is recompiled without version increment,
+        /// it is considered the same as before and won't be updated.
+        /// <para />
+        /// Assemblies are requested from remote nodes on demand.
+        /// For example, <see cref="IComputeFunc{TRes}"/> is sent to all nodes
+        /// via <see cref="ICompute.Broadcast{TRes}"/>. Each node then deserializes the instance and,
+        /// if containing assembly is not present, requests it from originating node (which did the
+        /// <see cref="ICompute.Broadcast{TRes}"/> call), if it is alive, or from any other node in cluster.
+        /// Therefore it is possible that eventually all nodes in cluster will have this assebly loaded.
+        /// </summary>
+        CurrentAppDomain
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/69876116/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 2a6f5f7..8fab8a4 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs
@@ -33,7 +33,9 @@ namespace Apache.Ignite.Core
     using Apache.Ignite.Core.Common;
     using Apache.Ignite.Core.Communication;
     using Apache.Ignite.Core.Communication.Tcp;
+    using Apache.Ignite.Core.Compute;
     using Apache.Ignite.Core.DataStructures.Configuration;
+    using Apache.Ignite.Core.Deployment;
     using Apache.Ignite.Core.Discovery;
     using Apache.Ignite.Core.Discovery.Tcp;
     using Apache.Ignite.Core.Events;
@@ -979,5 +981,19 @@ namespace Apache.Ignite.Core
         /// <see cref="MemoryConfiguration"/> for more details.
         /// </summary>
         public MemoryConfiguration MemoryConfiguration { get; set; }
+
+        /// <summary>
+        /// Gets or sets a value indicating how user assemblies should be loaded on remote nodes.
+        /// <para />
+        /// For example, when executing <see cref="ICompute.Call{TRes}(IComputeFunc{TRes})"/>,
+        /// the assembly with corresponding <see cref="IComputeFunc{TRes}"/> should be loaded on remote nodes.
+        /// With this option enabled, Ignite will attempt to send the assembly to remote nodes
+        /// and load it there automatically.
+        /// <para />
+        /// Default is <see cref="Apache.Ignite.Core.Deployment.PeerAssemblyLoadingMode.Disabled"/>.
+        /// <para />
+        /// Peer loading is enabled for <see cref="ICompute"/> functionality.
+        /// </summary>
+        public PeerAssemblyLoadingMode PeerAssemblyLoadingMode { get; set; }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/69876116/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 ad92661..bdfa8db 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
@@ -96,6 +96,13 @@
         </xs:restriction>
     </xs:simpleType>
 
+    <xs:simpleType name="peerAssemblyLoadingMode" final="restriction">
+        <xs:restriction base="xs:string">
+            <xs:enumeration value="Disabled" />
+            <xs:enumeration value="CurrentAppDomain" />
+        </xs:restriction>
+    </xs:simpleType>
+
     <xs:element name="igniteConfiguration">
         <xs:annotation>
             <xs:documentation>Ignite configuration root.</xs:documentation>
@@ -1411,6 +1418,11 @@
                     </xs:documentation>
                 </xs:annotation>
             </xs:attribute>
+            <xs:attribute name="peerAssemblyLoadingMode" type="peerAssemblyLoadingMode">
+                <xs:annotation>
+                    <xs:documentation>Indicates whether user assemblies should be loaded on remote nodes automatically.</xs:documentation>
+                </xs:annotation>
+            </xs:attribute>
         </xs:complexType>
     </xs:element>
 </xs:schema>

http://git-wip-us.apache.org/repos/asf/ignite/blob/69876116/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 fdddbb7..c5d056d 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Ignition.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Ignition.cs
@@ -84,7 +84,6 @@ namespace Apache.Ignite.Core
         [SuppressMessage("Microsoft.Performance", "CA1810:InitializeReferenceTypeStaticFieldsInline")]
         static Ignition()
         {
-            AppDomain.CurrentDomain.AssemblyResolve += CurrentDomain_AssemblyResolve;
             AppDomain.CurrentDomain.DomainUnload += CurrentDomain_DomainUnload;
         }
 
@@ -124,7 +123,7 @@ namespace Apache.Ignite.Core
         }
 
         /// <summary>
-        /// Reads <see cref="IgniteConfiguration"/> from application configuration 
+        /// Reads <see cref="IgniteConfiguration"/> from application configuration
         /// <see cref="IgniteConfigurationSection"/> with <see cref="ConfigurationSectionName"/>
         /// name and starts Ignite.
         /// </summary>
@@ -284,7 +283,7 @@ namespace Apache.Ignite.Core
                     if (_startup.Error != null)
                     {
                         // Wrap in a new exception to preserve original stack trace.
-                        throw new IgniteException("Failed to start Ignite.NET, check inner exception for details", 
+                        throw new IgniteException("Failed to start Ignite.NET, check inner exception for details",
                             _startup.Error);
                     }
 
@@ -722,18 +721,7 @@ namespace Apache.Ignite.Core
 
             GC.Collect();
         }
-        
-        /// <summary>
-        /// Handles the AssemblyResolve event of the CurrentDomain control.
-        /// </summary>
-        /// <param name="sender">The source of the event.</param>
-        /// <param name="args">The <see cref="ResolveEventArgs"/> instance containing the event data.</param>
-        /// <returns>Manually resolved assembly, or null.</returns>
-        private static Assembly CurrentDomain_AssemblyResolve(object sender, ResolveEventArgs args)
-        {
-            return LoadedAssembliesResolver.Instance.GetAssembly(args.Name);
-        }
-                
+
         /// <summary>
         /// Handles the DomainUnload event of the CurrentDomain control.
         /// </summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/69876116/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinarizableSerializer.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinarizableSerializer.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinarizableSerializer.cs
index 2273a93..a211360 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinarizableSerializer.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinarizableSerializer.cs
@@ -17,6 +17,7 @@
 
 namespace Apache.Ignite.Core.Impl.Binary
 {
+    using System;
     using System.Runtime.Serialization;
     using Apache.Ignite.Core.Binary;
 
@@ -38,9 +39,9 @@ namespace Apache.Ignite.Core.Impl.Binary
         }
 
         /** <inheritdoc /> */
-        public T ReadBinary<T>(BinaryReader reader, IBinaryTypeDescriptor desc, int pos)
+        public T ReadBinary<T>(BinaryReader reader, IBinaryTypeDescriptor desc, int pos, Type typeOverride)
         {
-            var obj = (T) FormatterServices.GetUninitializedObject(desc.Type);
+            var obj = (T) FormatterServices.GetUninitializedObject(typeOverride ?? desc.Type);
 
             reader.AddHandle(pos, obj);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/69876116/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryEqualityComparerSerializer.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryEqualityComparerSerializer.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryEqualityComparerSerializer.cs
deleted file mode 100644
index 5f28270..0000000
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryEqualityComparerSerializer.cs
+++ /dev/null
@@ -1 +0,0 @@
-
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/69876116/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryProcessor.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryProcessor.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryProcessor.cs
index f48bcc0..b8937c9 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryProcessor.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryProcessor.cs
@@ -232,7 +232,7 @@ namespace Apache.Ignite.Core.Impl.Binary
         }
 
         /// <summary>
-        /// Gets the type by id.
+        /// Gets the type name by id.
         /// </summary>
         /// <param name="id">The identifier.</param>
         /// <returns>Type or null.</returns>

http://git-wip-us.apache.org/repos/asf/ignite/blob/69876116/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 73a0456..e38085c 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReader.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReader.cs
@@ -522,13 +522,17 @@ namespace Apache.Ignite.Core.Impl.Binary
         /// <summary>
         /// Deserialize object.
         /// </summary>
+        /// <param name="typeOverride">The type override.
+        /// There can be multiple versions of the same type when peer assembly loading is enabled.
+        /// Only first one is registered in Marshaller.
+        /// This parameter specifies exact type to be instantiated.</param>
         /// <returns>Deserialized object.</returns>
-        public T Deserialize<T>()
+        public T Deserialize<T>(Type typeOverride = null)
         {
             T res;
 
             // ReSharper disable once CompareNonConstrainedGenericWithNull
-            if (!TryDeserialize(out res) && default(T) != null)
+            if (!TryDeserialize(out res, typeOverride) && default(T) != null)
                 throw new BinaryObjectException(string.Format("Invalid data on deserialization. " +
                     "Expected: '{0}' But was: null", typeof (T)));
 
@@ -538,8 +542,15 @@ namespace Apache.Ignite.Core.Impl.Binary
         /// <summary>
         /// Deserialize object.
         /// </summary>
-        /// <returns>Deserialized object.</returns>
-        public bool TryDeserialize<T>(out T res)
+        /// <param name="res">Deserialized object.</param>
+        /// <param name="typeOverride">The type override.
+        /// There can be multiple versions of the same type when peer assembly loading is enabled.
+        /// Only first one is registered in Marshaller.
+        /// This parameter specifies exact type to be instantiated.</param>
+        /// <returns>
+        /// Deserialized object.
+        /// </returns>
+        public bool TryDeserialize<T>(out T res, Type typeOverride = null)
         {
             int pos = Stream.Position;
 
@@ -557,12 +568,12 @@ namespace Apache.Ignite.Core.Impl.Binary
                     return false;
 
                 case BinaryUtils.HdrHnd:
-                    res = ReadHandleObject<T>(pos);
+                    res = ReadHandleObject<T>(pos, typeOverride);
 
                     return true;
 
                 case BinaryUtils.HdrFull:
-                    res = ReadFullObject<T>(pos);
+                    res = ReadFullObject<T>(pos, typeOverride);
 
                     return true;
 
@@ -583,7 +594,7 @@ namespace Apache.Ignite.Core.Impl.Binary
 
             throw new BinaryObjectException("Invalid header on deserialization [pos=" + pos + ", hdr=" + hdr + ']');
         }
-                
+
         /// <summary>
         /// Gets the flag indicating that there is custom type information in raw region.
         /// </summary>
@@ -657,8 +668,14 @@ namespace Apache.Ignite.Core.Impl.Binary
         /// <summary>
         /// Reads the full object.
         /// </summary>
+        /// <param name="pos">The position.</param>
+        /// <param name="typeOverride">The type override.
+        /// There can be multiple versions of the same type when peer assembly loading is enabled.
+        /// Only first one is registered in Marshaller.
+        /// This parameter specifies exact type to be instantiated.</param>
+        /// <returns>Resulting object</returns>
         [SuppressMessage("Microsoft.Performance", "CA1804:RemoveUnusedLocals", MessageId = "hashCode")]
-        private T ReadFullObject<T>(int pos)
+        private T ReadFullObject<T>(int pos, Type typeOverride)
         {
             var hdr = BinaryObjectHeader.Read(Stream, pos);
 
@@ -696,24 +713,17 @@ namespace Apache.Ignite.Core.Impl.Binary
                 {
                     // Find descriptor.
                     var desc = hdr.TypeId == BinaryUtils.TypeUnregistered
-                        ? _marsh.GetDescriptor(Type.GetType(ReadString(), true))
-                        : _marsh.GetDescriptor(hdr.IsUserType, hdr.TypeId, true);
+                        ? _marsh.GetDescriptor(ReadUnregisteredType(typeOverride))
+                        : _marsh.GetDescriptor(hdr.IsUserType, hdr.TypeId, true, null, typeOverride);
 
                     // Instantiate object. 
                     if (desc.Type == null)
                     {
-                        if (desc is BinarySurrogateTypeDescriptor)
-                        {
-                            throw new BinaryObjectException(string.Format(
-                                "Unknown type ID: {0}. " +
-                                "This usually indicates missing BinaryConfiguration. " +
-                                "Make sure that all nodes have the same BinaryConfiguration.", hdr.TypeId));
-                        }
-
                         throw new BinaryObjectException(string.Format(
                             "No matching type found for object [typeId={0}, typeName={1}]. " +
                             "This usually indicates that assembly with specified type is not loaded on a node. " +
-                            "When using Apache.Ignite.exe, make sure to load assemblies with -assembly parameter.",
+                            "When using Apache.Ignite.exe, make sure to load assemblies with -assembly parameter. " +
+                            "Alternatively, set IgniteConfiguration.PeerAssemblyLoadingEnabled to true.",
                             desc.TypeId, desc.TypeName));
                     }
 
@@ -728,7 +738,7 @@ namespace Apache.Ignite.Core.Impl.Binary
                     _frame.Raw = false;
 
                     // Read object.
-                    var obj = desc.Serializer.ReadBinary<T>(this, desc, pos);
+                    var obj = desc.Serializer.ReadBinary<T>(this, desc, pos, typeOverride);
 
                     _frame.Struct.UpdateReaderStructure();
 
@@ -746,6 +756,16 @@ namespace Apache.Ignite.Core.Impl.Binary
         }
 
         /// <summary>
+        /// Reads the unregistered type.
+        /// </summary>
+        private Type ReadUnregisteredType(Type knownType)
+        {
+            var typeName = ReadString();  // Must read always.
+
+            return knownType ?? Type.GetType(typeName, true);
+        }
+
+        /// <summary>
         /// Sets the current schema.
         /// </summary>
         private void SetCurSchema(IBinaryTypeDescriptor desc)
@@ -815,7 +835,7 @@ namespace Apache.Ignite.Core.Impl.Binary
         /// <summary>
         /// Reads the handle object.
         /// </summary>
-        private T ReadHandleObject<T>(int pos)
+        private T ReadHandleObject<T>(int pos, Type typeOverride)
         {
             // Get handle position.
             int hndPos = pos - Stream.ReadInt();
@@ -833,7 +853,7 @@ namespace Apache.Ignite.Core.Impl.Binary
                         // No such handler, i.e. we trying to deserialize inner object before deserializing outer.
                         Stream.Seek(hndPos, SeekOrigin.Begin);
 
-                        hndObj = Deserialize<T>();
+                        hndObj = Deserialize<T>(typeOverride);
                     }
 
                     // Notify builder that we deserialized object on other location.
@@ -977,7 +997,7 @@ namespace Apache.Ignite.Core.Impl.Binary
                 return default(T);
 
             if (hdr == BinaryUtils.HdrHnd)
-                return ReadHandleObject<T>(Stream.Position - 1);
+                return ReadHandleObject<T>(Stream.Position - 1, null);
 
             if (expHdr != hdr)
                 throw new BinaryObjectException(string.Format("Invalid header on deserialization. " +

http://git-wip-us.apache.org/repos/asf/ignite/blob/69876116/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReflectiveSerializerInternal.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReflectiveSerializerInternal.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReflectiveSerializerInternal.cs
index 69f7132..62c19ef 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReflectiveSerializerInternal.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReflectiveSerializerInternal.cs
@@ -84,13 +84,14 @@ namespace Apache.Ignite.Core.Impl.Binary
         }
 
         /** <inheritdoc /> */
-        T IBinarySerializerInternal.ReadBinary<T>(BinaryReader reader, IBinaryTypeDescriptor desc, int pos)
+        T IBinarySerializerInternal.ReadBinary<T>(BinaryReader reader, IBinaryTypeDescriptor desc, int pos,
+            Type typeOverride)
         {
             Debug.Assert(_rActions != null);
             Debug.Assert(reader != null);
             Debug.Assert(desc != null);
 
-            var obj = FormatterServices.GetUninitializedObject(desc.Type);
+            var obj = FormatterServices.GetUninitializedObject(typeOverride ?? desc.Type);
 
             var ctx = GetStreamingContext(reader);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/69876116/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinarySystemTypeSerializer.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinarySystemTypeSerializer.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinarySystemTypeSerializer.cs
index 1ea1f0b..681f57b 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinarySystemTypeSerializer.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinarySystemTypeSerializer.cs
@@ -48,7 +48,7 @@ namespace Apache.Ignite.Core.Impl.Binary
         }
 
         /** <inheritDoc /> */
-        public T1 ReadBinary<T1>(BinaryReader reader, IBinaryTypeDescriptor desc, int pos)
+        public T1 ReadBinary<T1>(BinaryReader reader, IBinaryTypeDescriptor desc, int pos, Type typeOverride)
         {
             return TypeCaster<T1>.Cast(_ctor(reader));
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/69876116/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 a5fed48..da2b371 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryWriter.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryWriter.cs
@@ -54,6 +54,9 @@ namespace Apache.Ignite.Core.Impl.Binary
         /** Whether we are currently detaching an object. */
         private bool _detaching;
 
+        /** Whether we are directly within peer loading object holder. */
+        private bool _isInWrapper;
+
         /** Schema holder. */
         private readonly BinaryObjectSchemaHolder _schema = BinaryObjectSchemaHolder.Current;
 
@@ -1165,6 +1168,22 @@ namespace Apache.Ignite.Core.Impl.Binary
                 return;
             }
 
+            // Wrap objects as required.
+            if (WrapperFunc != null && type != WrapperFunc.Method.ReturnType)
+            {
+                if (_isInWrapper)
+                {
+                    _isInWrapper = false;
+                }
+                else
+                {
+                    _isInWrapper = true;
+                    Write(WrapperFunc(obj));
+
+                    return;
+                }
+            }
+
             // Suppose that we faced normal object and perform descriptor lookup.
             var desc = _marsh.GetDescriptor(type);
 
@@ -1424,6 +1443,11 @@ namespace Apache.Ignite.Core.Impl.Binary
         }
 
         /// <summary>
+        /// Gets or sets a function to wrap all serializer objects.
+        /// </summary>
+        internal Func<object, object> WrapperFunc { get; set; }
+
+        /// <summary>
         /// Stream.
         /// </summary>
         internal IBinaryStream Stream

http://git-wip-us.apache.org/repos/asf/ignite/blob/69876116/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryWriterExtensions.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryWriterExtensions.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryWriterExtensions.cs
index b13a9ea..64bfa35 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryWriterExtensions.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryWriterExtensions.cs
@@ -18,6 +18,7 @@
 namespace Apache.Ignite.Core.Impl.Binary
 {
     using System;
+    using Apache.Ignite.Core.Binary;
 
     /// <summary>
     /// Writer extensions.
@@ -27,7 +28,7 @@ namespace Apache.Ignite.Core.Impl.Binary
         /// <summary>
         /// Writes the nullable boolean.
         /// </summary>
-        public static void WriteBooleanNullable(this BinaryWriter writer, bool? value)
+        public static void WriteBooleanNullable(this IBinaryRawWriter writer, bool? value)
         {
             if (value != null)
             {
@@ -41,7 +42,7 @@ namespace Apache.Ignite.Core.Impl.Binary
         /// <summary>
         /// Writes the nullable boolean.
         /// </summary>
-        public static void WriteIntNullable(this BinaryWriter writer, int? value)
+        public static void WriteIntNullable(this IBinaryRawWriter writer, int? value)
         {
             if (value != null)
             {
@@ -55,7 +56,7 @@ namespace Apache.Ignite.Core.Impl.Binary
         /// <summary>
         /// Writes the timespan.
         /// </summary>
-        public static void WriteTimeSpanAsLong(this BinaryWriter writer, TimeSpan value)
+        public static void WriteTimeSpanAsLong(this IBinaryRawWriter writer, TimeSpan value)
         {
             writer.WriteLong((long) value.TotalMilliseconds);
         }
@@ -63,7 +64,7 @@ namespace Apache.Ignite.Core.Impl.Binary
         /// <summary>
         /// Writes the nullable boolean.
         /// </summary>
-        public static void WriteTimeSpanAsLongNullable(this BinaryWriter writer, TimeSpan? value)
+        public static void WriteTimeSpanAsLongNullable(this IBinaryRawWriter writer, TimeSpan? value)
         {
             if (value != null)
             {

http://git-wip-us.apache.org/repos/asf/ignite/blob/69876116/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/IBinarySerializerInternal.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/IBinarySerializerInternal.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/IBinarySerializerInternal.cs
index b775999..42145e5 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/IBinarySerializerInternal.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/IBinarySerializerInternal.cs
@@ -17,6 +17,8 @@
 
 namespace Apache.Ignite.Core.Impl.Binary
 {
+    using System;
+
     /// <summary>
     /// Internal generic serializer interface.
     /// </summary>
@@ -25,12 +27,18 @@ namespace Apache.Ignite.Core.Impl.Binary
         /// <summary>
         /// Write binary object.
         /// </summary>
+        /// <param name="obj">The object.</param>
+        /// <param name="writer">The writer.</param>
         void WriteBinary<T>(T obj, BinaryWriter writer);
 
         /// <summary>
         /// Read binary object.
         /// </summary>
-        T ReadBinary<T>(BinaryReader reader, IBinaryTypeDescriptor desc, int pos);
+        /// <param name="reader">The reader.</param>
+        /// <param name="desc">The descriptor.</param>
+        /// <param name="pos">The position.</param>
+        /// <param name="typeOverride">Type override, can be null.</param>
+        T ReadBinary<T>(BinaryReader reader, IBinaryTypeDescriptor desc, int pos, Type typeOverride);
 
         /// <summary>
         /// Gets a value indicating whether this serializer supports handles.

http://git-wip-us.apache.org/repos/asf/ignite/blob/69876116/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Marshaller.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Marshaller.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Marshaller.cs
index 8f12acf..9634e27 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Marshaller.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Marshaller.cs
@@ -33,6 +33,7 @@ namespace Apache.Ignite.Core.Impl.Binary
     using Apache.Ignite.Core.Impl.Compute;
     using Apache.Ignite.Core.Impl.Compute.Closure;
     using Apache.Ignite.Core.Impl.Datastream;
+    using Apache.Ignite.Core.Impl.Deployment;
     using Apache.Ignite.Core.Impl.Messaging;
     using Apache.Ignite.Core.Log;
 
@@ -339,7 +340,7 @@ namespace Apache.Ignite.Core.Impl.Binary
                         IDictionary<int, BinaryTypeHolder> metas0 =
                             new Dictionary<int, BinaryTypeHolder>(_metas);
 
-                        holder = new BinaryTypeHolder(desc.TypeId, desc.TypeName, desc.AffinityKeyFieldName, 
+                        holder = new BinaryTypeHolder(desc.TypeId, desc.TypeName, desc.AffinityKeyFieldName,
                             desc.IsEnum, this);
 
                         metas0[desc.TypeId] = holder;
@@ -423,11 +424,12 @@ namespace Apache.Ignite.Core.Impl.Binary
         /// Only when we really deserialize the value, requiresType is set to true
         /// and we attempt to resolve the type by all means.</param>
         /// <param name="typeName">Known type name.</param>
+        /// <param name="knownType">Optional known type.</param>
         /// <returns>
         /// Descriptor.
         /// </returns>
-        public IBinaryTypeDescriptor GetDescriptor(bool userType, int typeId, bool requiresType = false, 
-            string typeName = null)
+        public IBinaryTypeDescriptor GetDescriptor(bool userType, int typeId, bool requiresType = false,
+            string typeName = null, Type knownType = null)
         {
             BinaryFullTypeDescriptor desc;
 
@@ -442,18 +444,29 @@ namespace Apache.Ignite.Core.Impl.Binary
             if (requiresType && _ignite != null)
             {
                 // Check marshaller context for dynamically registered type.
-                typeName = typeName ?? _ignite.BinaryProcessor.GetTypeName(typeId);
+                var type = knownType;
 
-                if (typeName != null)
+                if (type == null && _ignite != null)
                 {
-                    var type = new TypeResolver().ResolveType(typeName, nameMapper: 
-                        _cfg.NameMapper ?? GetDefaultNameMapper());
+                    typeName = typeName ?? _ignite.BinaryProcessor.GetTypeName(typeId);
 
-                    if (type != null)
+                    if (typeName != null)
                     {
-                        return AddUserType(type, typeId, GetTypeName(type), true, desc);
+                        type = new TypeResolver().ResolveType(typeName,
+                            nameMapper: _cfg.NameMapper ?? GetDefaultNameMapper());
+
+                        if (type == null)
+                        {
+                            // Type is registered, but assembly is not present.
+                            return new BinarySurrogateTypeDescriptor(_cfg, typeId, typeName);
+                        }
                     }
                 }
+
+                if (type != null)
+                {
+                    return AddUserType(type, typeId, GetTypeName(type), true, desc);
+                }
             }
 
             var meta = GetBinaryType(typeId);
@@ -593,7 +606,7 @@ namespace Apache.Ignite.Core.Impl.Binary
         /// <summary>
         /// Gets the serializer.
         /// </summary>
-        private static IBinarySerializerInternal GetSerializer(BinaryConfiguration cfg, 
+        private static IBinarySerializerInternal GetSerializer(BinaryConfiguration cfg,
             BinaryTypeConfiguration typeCfg, Type type, int typeId, IBinaryNameMapper nameMapper,
             IBinaryIdMapper idMapper, ILogger log)
         {
@@ -732,6 +745,10 @@ namespace Apache.Ignite.Core.Impl.Binary
             AddSystemType(BinaryUtils.TypePlatformJavaObjectFactoryProxy, r => new PlatformJavaObjectFactoryProxy());
             AddSystemType(0, r => new ObjectInfoHolder(r));
             AddSystemType(BinaryUtils.TypeIgniteUuid, r => new IgniteGuid(r));
+            AddSystemType(0, r => new GetAssemblyFunc());
+            AddSystemType(0, r => new AssemblyRequest(r));
+            AddSystemType(0, r => new AssemblyRequestResult(r));
+            AddSystemType<PeerLoadingObjectHolder>(0, null, serializer: new PeerLoadingObjectHolderSerializer());
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/69876116/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/SerializableSerializer.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/SerializableSerializer.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/SerializableSerializer.cs
index 1e4af4b..13310e4 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/SerializableSerializer.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/SerializableSerializer.cs
@@ -87,11 +87,13 @@ namespace Apache.Ignite.Core.Impl.Binary
         }
 
         /** <inheritdoc /> */
-        public T ReadBinary<T>(BinaryReader reader, IBinaryTypeDescriptor desc, int pos)
+        public T ReadBinary<T>(BinaryReader reader, IBinaryTypeDescriptor desc, int pos, Type typeOverride)
         {
             object res;
             var ctx = GetStreamingContext(reader);
 
+            var type = typeOverride ?? desc.Type;
+
             // Read additional information from raw part, if flag is set.
             IEnumerable<string> fieldNames;
             Type customType = null;
@@ -119,7 +121,7 @@ namespace Apache.Ignite.Core.Impl.Binary
                 if (customType != null)
                 {
                     // Custom type is present, which returns original type via IObjectReference.
-                    var serInfo = ReadSerializationInfo(reader, fieldNames, desc, dotNetFields);
+                    var serInfo = ReadSerializationInfo(reader, fieldNames, type, dotNetFields);
 
                     res = ReadAsCustomType(customType, serInfo, ctx);
 
@@ -131,7 +133,7 @@ namespace Apache.Ignite.Core.Impl.Binary
                 }
                 else
                 {
-                    res = FormatterServices.GetUninitializedObject(desc.Type);
+                    res = FormatterServices.GetUninitializedObject(type);
 
                     _serializableTypeDesc.OnDeserializing(res, ctx);
 
@@ -140,7 +142,7 @@ namespace Apache.Ignite.Core.Impl.Binary
                     reader.AddHandle(pos, res);
 
                     // Read actual data and call constructor.
-                    var serInfo = ReadSerializationInfo(reader, fieldNames, desc, dotNetFields);
+                    var serInfo = ReadSerializationInfo(reader, fieldNames, type, dotNetFields);
                     _serializableTypeDesc.SerializationCtorUninitialized(res, serInfo, ctx);
                 }
 
@@ -536,9 +538,9 @@ namespace Apache.Ignite.Core.Impl.Binary
         /// Reads the serialization information.
         /// </summary>
         private static SerializationInfo ReadSerializationInfo(BinaryReader reader, 
-            IEnumerable<string> fieldNames, IBinaryTypeDescriptor desc, ICollection<int> dotNetFields)
+            IEnumerable<string> fieldNames, Type type, ICollection<int> dotNetFields)
         {
-            var serInfo = new SerializationInfo(desc.Type, new FormatterConverter());
+            var serInfo = new SerializationInfo(type, new FormatterConverter());
 
             if (dotNetFields == null)
             {

http://git-wip-us.apache.org/repos/asf/ignite/blob/69876116/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/TypeResolver.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/TypeResolver.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/TypeResolver.cs
index fa59d62..6fa7918 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/TypeResolver.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/TypeResolver.cs
@@ -68,7 +68,7 @@ namespace Apache.Ignite.Core.Impl.Binary
         /// <param name="typeName">Name of the type.</param>
         /// <param name="assemblies">Assemblies to look in.</param>
         /// <param name="nameMapper">The name mapper.</param>
-        /// <returns> 
+        /// <returns>
         /// Resolved type. 
         /// </returns>
         private static Type ResolveType(string assemblyName, TypeNameParser typeName, ICollection<Assembly> assemblies,
@@ -144,7 +144,7 @@ namespace Apache.Ignite.Core.Impl.Binary
         /// <param name="assemblies">The assemblies.</param>
         /// <param name="nameMapper">The name mapper.</param>
         /// <returns>Resolved type, or null.</returns>
-        private static Type ResolveNonGenericType(string assemblyName, string typeName, 
+        private static Type ResolveNonGenericType(string assemblyName, string typeName,
             ICollection<Assembly> assemblies, IBinaryNameMapper nameMapper)
         {
             // Fully-qualified name can be resolved with system mechanism.
@@ -187,7 +187,7 @@ namespace Apache.Ignite.Core.Impl.Binary
 
             try
             {
-                var result = ResolveType(assemblyName, typeName, GetNotLoadedReferencedAssemblies().ToArray(), 
+                var result = ResolveType(assemblyName, typeName, GetNotLoadedReferencedAssemblies().ToArray(),
                     nameMapper);
 
                 if (result == null)

http://git-wip-us.apache.org/repos/asf/ignite/blob/69876116/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/UserSerializerProxy.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/UserSerializerProxy.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/UserSerializerProxy.cs
index b0d393d..95762fd 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/UserSerializerProxy.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/UserSerializerProxy.cs
@@ -17,6 +17,7 @@
 
 namespace Apache.Ignite.Core.Impl.Binary
 {
+    using System;
     using System.Diagnostics;
     using System.Runtime.Serialization;
     using Apache.Ignite.Core.Binary;
@@ -47,9 +48,9 @@ namespace Apache.Ignite.Core.Impl.Binary
         }
 
         /** <inheritdoc /> */
-        public T ReadBinary<T>(BinaryReader reader, IBinaryTypeDescriptor desc, int pos)
+        public T ReadBinary<T>(BinaryReader reader, IBinaryTypeDescriptor desc, int pos, Type typeOverride)
         {
-            var obj = FormatterServices.GetUninitializedObject(desc.Type);
+            var obj = FormatterServices.GetUninitializedObject(typeOverride ?? desc.Type);
 
             reader.AddHandle(pos, obj);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/69876116/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/CopyOnWriteConcurrentDictionary.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/CopyOnWriteConcurrentDictionary.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/CopyOnWriteConcurrentDictionary.cs
index 78cb8b6..cd26af7 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/CopyOnWriteConcurrentDictionary.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/CopyOnWriteConcurrentDictionary.cs
@@ -96,13 +96,5 @@ namespace Apache.Ignite.Core.Impl.Common
         {
             return _dict.ContainsKey(key);
         }
-
-        /// <summary>
-        /// Gets the values.
-        /// </summary>
-        public ICollection<TValue> Values
-        {
-            get { return _dict.Values; }
-        }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/69876116/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/LoadedAssembliesResolver.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/LoadedAssembliesResolver.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/LoadedAssembliesResolver.cs
index 9628478..06dffb7 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/LoadedAssembliesResolver.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/LoadedAssembliesResolver.cs
@@ -19,6 +19,7 @@ namespace Apache.Ignite.Core.Impl.Common
 {
     using System;
     using System.Collections.Generic;
+    using System.Diagnostics;
     using System.Diagnostics.CodeAnalysis;
     using System.Reflection;
 
@@ -29,7 +30,8 @@ namespace Apache.Ignite.Core.Impl.Common
     internal class LoadedAssembliesResolver
     {
         // The lazy singleton instance.
-        private static readonly Lazy<LoadedAssembliesResolver> LazyInstance = new Lazy<LoadedAssembliesResolver>();
+        private static readonly Lazy<LoadedAssembliesResolver> LazyInstance =
+            new Lazy<LoadedAssembliesResolver>(() => new LoadedAssembliesResolver());
 
         // Assemblies map.
         private volatile Dictionary<string, Assembly> _map;
@@ -37,7 +39,7 @@ namespace Apache.Ignite.Core.Impl.Common
         /// <summary>
         /// Initializes a new instance of the <see cref="LoadedAssembliesResolver"/> class.
         /// </summary>
-        public LoadedAssembliesResolver()
+        private LoadedAssembliesResolver()
         {
             lock (this)
             {
@@ -89,6 +91,8 @@ namespace Apache.Ignite.Core.Impl.Common
         [SuppressMessage("ReSharper", "InconsistentlySynchronizedField")]
         public Assembly GetAssembly(string assemblyName)
         {
+            Debug.Assert(!string.IsNullOrWhiteSpace(assemblyName));
+
             Assembly asm;
 
             return _map.TryGetValue(assemblyName, out asm) ? asm : null;