You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2017/05/31 07:35:32 UTC

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

Repository: ignite
Updated Branches:
  refs/heads/ignite-5075 3efd652c6 -> 08ed6da98


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; }
+        }
+    }
+}


[07/19] ignite git commit: .NET: Fix build warnings

Posted by sb...@apache.org.
.NET: Fix build warnings


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

Branch: refs/heads/ignite-5075
Commit: 7adf588f0b85d4a3a0a210493f28b1f51f4336b9
Parents: cee5dd8
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Mon May 29 20:05:38 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Mon May 29 20:05:38 2017 +0300

----------------------------------------------------------------------
 modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Binary.cs | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7adf588f/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Binary.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Binary.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Binary.cs
index bcf6c2d..8f75034 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Binary.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Binary.cs
@@ -198,6 +198,7 @@ namespace Apache.Ignite.Core.Impl.Binary
         /** <inheritDoc /> */
         public IBinaryObject BuildEnum(Type type, string valueName)
         {
+            IgniteArgumentCheck.NotNull(type, "type");
             IgniteArgumentCheck.NotNullOrEmpty(valueName, "valueName");
 
             var desc = Marshaller.GetDescriptor(type);


[08/19] ignite git commit: .NET: Improve async exception propagation

Posted by sb...@apache.org.
.NET: Improve async exception propagation


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

Branch: refs/heads/ignite-5075
Commit: 03b383c81bf2c7b7466164c849ac1a806d4afc3e
Parents: 7adf588
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Mon May 29 20:39:31 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Mon May 29 20:39:31 2017 +0300

----------------------------------------------------------------------
 .../Compute/ComputeApiTest.cs                     | 18 ++++++++++++------
 .../Apache.Ignite.Core/Impl/Common/Future.cs      |  5 ++---
 2 files changed, 14 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/03b383c8/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTest.cs
index 1d8ceb9..e4fd853 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTest.cs
@@ -1285,9 +1285,9 @@ namespace Apache.Ignite.Core.Tests.Compute
         [Test]
         public void TestExceptions()
         {
-            Assert.Throws<BinaryObjectException>(() => _grid1.GetCompute().Broadcast(new InvalidComputeAction()));
+            Assert.Throws<IgniteException>(() => _grid1.GetCompute().Broadcast(new InvalidComputeAction()));
 
-            Assert.Throws<BinaryObjectException>(
+            Assert.Throws<IgniteException>(
                 () => _grid1.GetCompute().Execute<NetSimpleJobArgument, NetSimpleJobResult, NetSimpleTaskResult>(
                     typeof (NetSimpleTask), new NetSimpleJobArgument(-1)));
 
@@ -1295,17 +1295,23 @@ namespace Apache.Ignite.Core.Tests.Compute
             var ex = Assert.Throws<IgniteException>(() =>
                 _grid1.GetCluster().ForLocal().GetCompute().Broadcast(new ExceptionalComputeAction()));
 
-            Assert.AreEqual("Compute job has failed on local node, examine InnerException for details.", ex.Message);
+            Assert.AreEqual("Async operation has failed, examine InnerException for details.", ex.Message);
             Assert.IsNotNull(ex.InnerException);
-            Assert.AreEqual(ExceptionalComputeAction.ErrorText, ex.InnerException.Message);
+            Assert.AreEqual("Compute job has failed on local node, examine InnerException for details.", 
+                ex.InnerException.Message);
+            Assert.IsNotNull(ex.InnerException.InnerException);
+            Assert.AreEqual(ExceptionalComputeAction.ErrorText, ex.InnerException.InnerException.Message);
 
             // Remote.
             ex = Assert.Throws<IgniteException>(() =>
                 _grid1.GetCluster().ForRemotes().GetCompute().Broadcast(new ExceptionalComputeAction()));
 
-            Assert.AreEqual("Compute job has failed on remote node, examine InnerException for details.", ex.Message);
+            Assert.AreEqual("Async operation has failed, examine InnerException for details.", ex.Message);
             Assert.IsNotNull(ex.InnerException);
-            Assert.AreEqual(ExceptionalComputeAction.ErrorText, ex.InnerException.Message);
+            Assert.AreEqual("Compute job has failed on remote node, examine InnerException for details.",
+                ex.InnerException.Message);
+            Assert.IsNotNull(ex.InnerException.InnerException);
+            Assert.AreEqual(ExceptionalComputeAction.ErrorText, ex.InnerException.InnerException.Message);
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/03b383c8/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/Future.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/Future.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/Future.cs
index bfdf5cb..b69ad56 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/Future.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/Future.cs
@@ -60,10 +60,9 @@ namespace Apache.Ignite.Core.Impl.Common
             }
             catch (AggregateException ex)
             {
-                if (ex.InnerException != null)
-                    throw ex.InnerException;
+                var innerEx = ex.InnerExceptions.Count > 1 ? ex : ex.InnerException;
 
-                throw;
+                throw new IgniteException("Async operation has failed, examine InnerException for details.", innerEx);
             }
         }
 


[18/19] ignite git commit: Merge remote-tracking branch 'remotes/origin/master' into ignite-5075

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/master' into ignite-5075

# Conflicts:
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java


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

Branch: refs/heads/ignite-5075
Commit: b74b739755d62c3f5104bfdb65fa7d0319712e28
Parents: 3efd652 10a0a84
Author: sboikov <sb...@gridgain.com>
Authored: Wed May 31 10:35:08 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed May 31 10:35:08 2017 +0300

----------------------------------------------------------------------
 .../jdbc2/JdbcAbstractDmlStatementSelfTest.java |  50 +-
 .../JdbcAbstractUpdateStatementSelfTest.java    |  11 +-
 .../ignite/internal/jdbc2/JdbcBlobTest.java     | 485 +++++++++++++++++++
 .../jdbc2/JdbcInsertStatementSelfTest.java      |  16 +-
 .../jdbc2/JdbcMergeStatementSelfTest.java       |  16 +-
 .../jdbc2/JdbcPreparedStatementSelfTest.java    |  47 ++
 .../jdbc/suite/IgniteJdbcDriverTestSuite.java   |   7 +-
 .../org/apache/ignite/internal/IgniteEx.java    |   9 +-
 .../apache/ignite/internal/IgniteKernal.java    |  19 +-
 .../apache/ignite/internal/jdbc2/JdbcBlob.java  | 191 ++++++++
 .../ignite/internal/jdbc2/JdbcConnection.java   |   2 +-
 .../internal/jdbc2/JdbcPreparedStatement.java   |   4 +-
 .../ignite/internal/jdbc2/JdbcResultSet.java    |   8 +-
 .../internal/processors/cache/CacheData.java    |  13 +
 .../cache/CacheJoinNodeDiscoveryData.java       |  15 +-
 .../processors/cache/ClusterCachesInfo.java     |  47 +-
 .../cache/DynamicCacheChangeRequest.java        |  21 +-
 .../cache/DynamicCacheDescriptor.java           |  13 +
 .../processors/cache/GridCacheAttributes.java   |  14 +-
 .../processors/cache/GridCacheProcessor.java    |  53 +-
 .../processors/cache/IgniteCacheProxy.java      |   2 +-
 .../datastructures/DataStructuresProcessor.java |   1 +
 .../datastructures/GridCacheLockImpl.java       |  10 +-
 .../processors/query/GridQueryProcessor.java    |   6 +-
 .../internal/processors/query/QueryUtils.java   |  14 +-
 .../handlers/cache/GridCacheCommandHandler.java |   2 +-
 .../ignite/spi/discovery/DiscoverySpi.java      |   9 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |   8 +-
 .../cache/IgniteDynamicCacheStartSelfTest.java  |   4 +-
 .../processors/igfs/IgfsIgniteMock.java         |   4 +-
 .../spi/discovery/tcp/TcpDiscoverySelfTest.java |   4 +-
 .../junits/multijvm/IgniteProcessProxy.java     |   4 +-
 .../cache/index/H2DynamicTableSelfTest.java     |  79 ++-
 .../query/IgniteSqlSchemaIndexingTest.java      |  18 +-
 .../query/IgniteSqlSplitterSelfTest.java        |   2 -
 modules/platforms/cpp/binary/Makefile.am        |  67 +--
 .../platforms/cpp/binary/include/Makefile.am    |  55 +--
 .../binary/include/ignite/binary/binary_type.h  |   2 +-
 .../ignite/impl/binary/binary_type_impl.h       |  61 +++
 .../ignite/impl/binary/binary_writer_impl.h     |   4 +-
 .../cpp/binary/project/vs/binary.vcxproj        |   2 +
 .../binary/project/vs/binary.vcxproj.filters    |   6 +
 .../binary/src/impl/binary/binary_type_impl.cpp |  61 +++
 modules/platforms/cpp/common/Makefile.am        |  54 +--
 .../platforms/cpp/common/include/Makefile.am    |   1 +
 .../common/include/ignite/common/cancelable.h   |  65 +++
 .../cpp/common/include/ignite/common/promise.h  |  18 +
 .../common/include/ignite/common/shared_state.h |  51 ++
 .../cpp/common/include/ignite/future.h          |  60 ++-
 .../cpp/common/include/ignite/ignite_error.h    |   3 +
 .../cpp/common/project/vs/common.vcxproj        |   1 +
 .../common/project/vs/common.vcxproj.filters    |   3 +
 modules/platforms/cpp/core-test/Makefile.am     | 101 ++--
 .../cpp/core-test/include/ignite/test_utils.h   |  24 +-
 .../cpp/core-test/project/vs/core-test.vcxproj  |   1 +
 .../project/vs/core-test.vcxproj.filters        |  10 +-
 .../cpp/core-test/src/cache_invoke_test.cpp     |   2 +-
 .../cpp/core-test/src/cluster_test.cpp          |   2 +-
 .../cpp/core-test/src/compute_test.cpp          | 337 +++++++++++++
 .../cpp/core-test/src/continuous_query_test.cpp |   2 +-
 .../platforms/cpp/core-test/src/test_utils.cpp  |   9 +
 modules/platforms/cpp/core/Makefile.am          |  82 ++--
 modules/platforms/cpp/core/include/Makefile.am  | 107 ++--
 .../cpp/core/include/ignite/cache/cache.h       |   4 +-
 .../cpp/core/include/ignite/cache/query/query.h |   1 -
 .../cpp/core/include/ignite/compute/compute.h   | 136 ++++++
 .../core/include/ignite/compute/compute_func.h  |  65 +++
 .../platforms/cpp/core/include/ignite/ignite.h  |  19 +-
 .../cpp/core/include/ignite/ignite_binding.h    |  25 +
 .../include/ignite/ignite_binding_context.h     |   2 +-
 .../cpp/core/include/ignite/impl/bindings.h     |  24 +
 .../ignite/impl/cluster/cluster_group_impl.h    |  31 +-
 .../ignite/impl/compute/cancelable_impl.h       |  70 +++
 .../include/ignite/impl/compute/compute_impl.h  | 118 +++++
 .../ignite/impl/compute/compute_job_holder.h    | 139 ++++++
 .../ignite/impl/compute/compute_job_result.h    | 161 ++++++
 .../ignite/impl/compute/compute_task_holder.h   | 213 ++++++++
 .../include/ignite/impl/ignite_binding_impl.h   |   2 +
 .../include/ignite/impl/ignite_environment.h    |  69 +++
 .../cpp/core/include/ignite/impl/ignite_impl.h  |  23 +-
 .../ignite/impl/interop/interop_target.h        |  11 +-
 .../platforms/cpp/core/project/vs/core.vcxproj  |   9 +
 .../cpp/core/project/vs/core.vcxproj.filters    |  33 ++
 modules/platforms/cpp/core/src/ignite.cpp       |   5 +
 .../src/impl/cluster/cluster_group_impl.cpp     |  23 +-
 .../core/src/impl/compute/cancelable_impl.cpp   |  59 +++
 .../cpp/core/src/impl/compute/compute_impl.cpp  |  35 ++
 .../cpp/core/src/impl/ignite_environment.cpp    | 260 +++++++++-
 .../platforms/cpp/core/src/impl/ignite_impl.cpp |  15 +-
 .../core/src/impl/interop/interop_target.cpp    |  20 +
 .../src/impl/transactions/transactions_impl.cpp |  10 -
 modules/platforms/cpp/examples/Makefile.am      |   2 +-
 modules/platforms/cpp/examples/configure.ac     |   2 +-
 .../cpp/examples/odbc-example/Makefile.am       |  12 +-
 .../cpp/examples/project/vs/ignite-examples.sln |  12 +-
 .../cpp/examples/put-get-example/Makefile.am    |  58 +++
 .../put-get-example/config/example-cache.xml    |  67 +++
 .../project/vs/put-get-example.vcxproj          | 110 +++++
 .../project/vs/put-get-example.vcxproj.filters  |  38 ++
 .../put-get-example/src/put_get_example.cpp     | 126 +++++
 .../cpp/examples/putget-example/Makefile.am     |  58 ---
 .../putget-example/config/example-cache.xml     |  67 ---
 .../project/vs/putget-example.vcxproj           | 110 -----
 .../project/vs/putget-example.vcxproj.filters   |  38 --
 .../putget-example/src/putget_example.cpp       | 126 -----
 .../cpp/examples/query-example/Makefile.am      |  12 +-
 .../platforms/cpp/jni/include/ignite/jni/java.h |   1 +
 modules/platforms/cpp/jni/src/java.cpp          |  10 +
 .../StartupTest.cs                              |  78 +++
 .../Apache.Ignite.Core.Tests.NuGet/TestUtil.cs  |  30 ++
 .../Apache.Ignite.Core.Tests.csproj             |   9 +
 .../Binary/TypeNameParserTest.cs                |   8 +-
 .../Compute/BinarizableClosureTaskTest.cs       |   7 +-
 .../Compute/ClosureTaskTest.cs                  |   6 +-
 .../Compute/ComputeApiTest.cs                   |  20 +-
 .../Compute/IgniteExceptionTaskSelfTest.cs      |   8 +-
 .../Compute/SerializableClosureTaskTest.cs      |   5 +-
 .../Deployment/GetAddressFunc.cs                |  35 ++
 .../PeerAssemblyLoadingAllApisTest.cs           | 167 +++++++
 .../Deployment/PeerAssemblyLoadingTest.cs       | 192 ++++++++
 .../PeerAssemblyLoadingVersioningTest.cs        | 164 +++++++
 .../Deployment/ProcessNameFunc.cs               |  50 ++
 .../Deployment/ProcessNameTask.cs               |  74 +++
 .../Deployment/peer_assembly_app.config         |  35 ++
 .../IgniteConfigurationSerializerTest.cs        |  10 +-
 .../Log/CustomLoggerTest.cs                     |   5 +-
 .../Process/IgniteProcess.cs                    |  12 +-
 .../Apache.Ignite.Core.csproj                   |   9 +
 .../Apache.Ignite.Core.nuspec                   |   7 +-
 .../Cache/Configuration/QueryEntity.cs          |   4 +-
 .../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 +-
 .../Apache.Ignite.Core/Impl/Binary/Binary.cs    |   1 +
 .../Binary/BinaryEqualityComparerSerializer.cs  |   1 -
 .../Impl/Binary/BinaryProcessor.cs              |   2 +-
 .../Impl/Binary/BinaryReader.cs                 |  66 ++-
 .../Impl/Binary/BinaryReflectiveActions.cs      |   7 +-
 .../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 -
 .../Apache.Ignite.Core/Impl/Common/Future.cs    |  13 +-
 .../Impl/Common/IgniteHome.cs                   |   3 +-
 .../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 ++
 modules/platforms/dotnet/Apache.Ignite.FxCop    |   3 +-
 .../platforms/dotnet/Apache.Ignite/App.config   |  11 +
 modules/spring/pom.xml                          |   7 +
 .../apache/ignite/cache/spring/SpringCache.java |  60 ++-
 .../ignite/cache/spring/SpringCacheManager.java |  51 +-
 .../GridSpringCacheManagerMultiJvmSelfTest.java | 134 +++++
 .../GridSpringDynamicCacheTestService.java      |  13 +
 .../ignite/cache/spring/SpringCacheTest.java    |   2 +-
 .../ignite/cache/spring/spring-caching1.xml     |  56 +++
 .../ignite/cache/spring/spring-caching2.xml     |  56 +++
 177 files changed, 6265 insertions(+), 967 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b74b7397/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheData.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/b74b7397/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
index 287a30d,7ff5622..1978f9e
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
@@@ -117,9 -112,7 +117,9 @@@ class ClusterCachesInfo 
                  CacheData cacheData = gridData.caches().get(locCfg.getName());
  
                  if (cacheData != null)
-                     checkCache(locCfg, cacheData.cacheConfiguration(), cacheData.receivedFrom());
+                     checkCache(locCacheInfo, cacheData, cacheData.receivedFrom());
 +
 +                validateStartCacheConfiguration(locCfg);
              }
          }
  
@@@ -142,9 -136,9 +143,12 @@@
          CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "cacheMode", "Cache mode",
              locAttr.cacheMode(), rmtAttr.cacheMode(), true);
  
 +        CU.checkAttributeMismatch(log, rmtCfg.getGroupName(), rmt, "groupName", "Cache group name",
 +            locCfg.getGroupName(), rmtCfg.getGroupName(), true);
 +
+         CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "sql", "SQL flag",
+             locAttr.sql(), rmtAttr.sql(), true);
+ 
          if (rmtAttr.cacheMode() != LOCAL) {
              CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "interceptor", "Cache Interceptor",
                  locAttr.interceptorClassName(), rmtAttr.interceptorClassName(), true);
@@@ -155,13 -149,10 +159,13 @@@
              CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "cachePreloadMode",
                  "Cache preload mode", locAttr.cacheRebalanceMode(), rmtAttr.cacheRebalanceMode(), true);
  
 +            CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "topologyValidator",
 +                "Cache topology validator", locAttr.topologyValidatorClassName(), rmtAttr.topologyValidatorClassName(), true);
 +
              ClusterNode rmtNode = ctx.discovery().node(rmt);
  
-             if (CU.affinityNode(ctx.discovery().localNode(), locCfg.getNodeFilter())
-                 && rmtNode != null && CU.affinityNode(rmtNode, rmtCfg.getNodeFilter())) {
+             if (CU.affinityNode(ctx.discovery().localNode(), locInfo.config().getNodeFilter())
+                 && rmtNode != null && CU.affinityNode(rmtNode, rmtData.cacheConfiguration().getNodeFilter())) {
                  CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "storeFactory", "Store factory",
                      locAttr.storeFactoryClassName(), rmtAttr.storeFactoryClassName(), true);
              }
@@@ -397,10 -380,28 +403,26 @@@
                  }
              }
              else if (req.stop()) {
 -                assert req.stop() ^ req.close() : req;
 -
                  if (desc != null) {
+                     if (req.sql() && !desc.sql()) {
+                         ctx.cache().completeCacheStartFuture(req, false,
+                             new IgniteCheckedException("Only cache created with CREATE TABLE may be removed with " +
+                                 "DROP TABLE [cacheName=" + req.cacheName() + ']'));
+ 
+                         continue;
+                     }
+ 
+                     if (!req.sql() && desc.sql()) {
+                         ctx.cache().completeCacheStartFuture(req, false,
+                             new IgniteCheckedException("Only cache created with cache API may be removed with " +
+                                 "direct call to destroyCache [cacheName=" + req.cacheName() + ']'));
+ 
+                         continue;
+                     }
+ 
                      DynamicCacheDescriptor old = registeredCaches.remove(req.cacheName());
  
 -                    assert old != null : "Dynamic cache map was concurrently modified [req=" + req + ']';
 +                    assert old != null && old == desc : "Dynamic cache map was concurrently modified [req=" + req + ']';
  
                      ctx.discovery().removeCacheFilter(req.cacheName());
  

http://git-wip-us.apache.org/repos/asf/ignite/blob/b74b7397/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheDescriptor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/b74b7397/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
index a9ad353,4af2518..ed695dc
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
@@@ -273,20 -278,13 +278,27 @@@ public class GridCacheAttributes implem
      }
  
      /**
 +     * @return Node filter class name.
 +     */
 +    String nodeFilterClassName() {
 +        return className(ccfg.getNodeFilter());
 +    }
 +
 +    /**
 +     * @return Topology validator class name.
 +     */
 +    String topologyValidatorClassName() {
 +        return className(ccfg.getTopologyValidator());
 +    }
 +
 +    /**
+      * @return SQL flag.
+      */
+     public boolean sql() {
+         return sql;
+     }
+ 
+     /**
       * @param obj Object to get class of.
       * @return Class name or {@code null}.
       */

http://git-wip-us.apache.org/repos/asf/ignite/blob/b74b7397/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/b74b7397/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/b74b7397/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
----------------------------------------------------------------------


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

Posted by sb...@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/ignite-5075
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;


[15/19] ignite git commit: IGNITE-5338 .NET: Fix TestMultipleAssembliesIndirectDependencyMultiLevel

Posted by sb...@apache.org.
IGNITE-5338 .NET: Fix TestMultipleAssembliesIndirectDependencyMultiLevel

This closes #2033


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

Branch: refs/heads/ignite-5075
Commit: 73ae278d87bf5d3f404719a4a3c03f88badbe7d3
Parents: fd1d87b
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Tue May 30 16:58:33 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Tue May 30 16:58:33 2017 +0300

----------------------------------------------------------------------
 .../Deployment/PeerAssemblyLoadingTest.cs                    | 8 ++++----
 1 file changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/73ae278d/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
index de1ecd6..27a2d65 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Deployment/PeerAssemblyLoadingTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Deployment/PeerAssemblyLoadingTest.cs
@@ -27,7 +27,7 @@ namespace Apache.Ignite.Core.Tests.Deployment
     using Apache.Ignite.Core.Impl;
     using Apache.Ignite.Core.Impl.Common;
     using Apache.Ignite.Core.Tests.Process;
-    using Apache.Ignite.Log4Net;
+    using Apache.Ignite.NLog;
     using NUnit.Framework;
     using Address = ExamplesDll::Apache.Ignite.ExamplesDll.Binary.Address;
 
@@ -111,9 +111,9 @@ namespace Apache.Ignite.Core.Tests.Deployment
         {
             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()}));
+                // Arg is object, value is from Apache.Ignite.Log4Net, and it further depends on NLog.
+                Assert.AreEqual("Apache.IgniteApache.Ignite.NLog.IgniteNLogLogger", remoteCompute.Call(
+                    new ProcessNameFunc {Arg = new IgniteNLogLogger()}));
             });
         }
 


[05/19] ignite git commit: IGNITE-3355: Implemented Compute::Call() for C++

Posted by sb...@apache.org.
IGNITE-3355: Implemented Compute::Call() for C++


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

Branch: refs/heads/ignite-5075
Commit: f9c96de578c8a87e8f2f7fee01d3719d0bb21359
Parents: 0f8af13
Author: Igor Sapego <is...@gridgain.com>
Authored: Mon May 29 17:29:10 2017 +0300
Committer: Igor Sapego <is...@gridgain.com>
Committed: Mon May 29 17:29:10 2017 +0300

----------------------------------------------------------------------
 modules/platforms/cpp/binary/Makefile.am        |  67 ++--
 .../platforms/cpp/binary/include/Makefile.am    |  55 +--
 .../binary/include/ignite/binary/binary_type.h  |   2 +-
 .../ignite/impl/binary/binary_type_impl.h       |  61 ++++
 .../ignite/impl/binary/binary_writer_impl.h     |   4 +-
 .../cpp/binary/project/vs/binary.vcxproj        |   2 +
 .../binary/project/vs/binary.vcxproj.filters    |   6 +
 .../binary/src/impl/binary/binary_type_impl.cpp |  61 ++++
 modules/platforms/cpp/common/Makefile.am        |  54 +--
 .../platforms/cpp/common/include/Makefile.am    |   1 +
 .../common/include/ignite/common/cancelable.h   |  65 ++++
 .../cpp/common/include/ignite/common/promise.h  |  18 +
 .../common/include/ignite/common/shared_state.h |  51 +++
 .../cpp/common/include/ignite/future.h          |  60 +++-
 .../cpp/common/include/ignite/ignite_error.h    |   3 +
 .../cpp/common/project/vs/common.vcxproj        |   1 +
 .../common/project/vs/common.vcxproj.filters    |   3 +
 modules/platforms/cpp/core-test/Makefile.am     | 101 +++---
 .../cpp/core-test/include/ignite/test_utils.h   |  24 +-
 .../cpp/core-test/project/vs/core-test.vcxproj  |   1 +
 .../project/vs/core-test.vcxproj.filters        |  10 +-
 .../cpp/core-test/src/cache_invoke_test.cpp     |   2 +-
 .../cpp/core-test/src/cluster_test.cpp          |   2 +-
 .../cpp/core-test/src/compute_test.cpp          | 337 +++++++++++++++++++
 .../cpp/core-test/src/continuous_query_test.cpp |   2 +-
 .../platforms/cpp/core-test/src/test_utils.cpp  |   9 +
 modules/platforms/cpp/core/Makefile.am          |  82 ++---
 modules/platforms/cpp/core/include/Makefile.am  | 107 +++---
 .../cpp/core/include/ignite/cache/cache.h       |   4 +-
 .../cpp/core/include/ignite/cache/query/query.h |   1 -
 .../cpp/core/include/ignite/compute/compute.h   | 136 ++++++++
 .../core/include/ignite/compute/compute_func.h  |  65 ++++
 .../platforms/cpp/core/include/ignite/ignite.h  |  19 +-
 .../cpp/core/include/ignite/ignite_binding.h    |  25 ++
 .../include/ignite/ignite_binding_context.h     |   2 +-
 .../cpp/core/include/ignite/impl/bindings.h     |  24 ++
 .../ignite/impl/cluster/cluster_group_impl.h    |  31 +-
 .../ignite/impl/compute/cancelable_impl.h       |  70 ++++
 .../include/ignite/impl/compute/compute_impl.h  | 118 +++++++
 .../ignite/impl/compute/compute_job_holder.h    | 139 ++++++++
 .../ignite/impl/compute/compute_job_result.h    | 161 +++++++++
 .../ignite/impl/compute/compute_task_holder.h   | 213 ++++++++++++
 .../include/ignite/impl/ignite_binding_impl.h   |   2 +
 .../include/ignite/impl/ignite_environment.h    |  69 ++++
 .../cpp/core/include/ignite/impl/ignite_impl.h  |  23 +-
 .../ignite/impl/interop/interop_target.h        |  11 +-
 .../platforms/cpp/core/project/vs/core.vcxproj  |   9 +
 .../cpp/core/project/vs/core.vcxproj.filters    |  33 ++
 modules/platforms/cpp/core/src/ignite.cpp       |   5 +
 .../src/impl/cluster/cluster_group_impl.cpp     |  23 +-
 .../core/src/impl/compute/cancelable_impl.cpp   |  59 ++++
 .../cpp/core/src/impl/compute/compute_impl.cpp  |  35 ++
 .../cpp/core/src/impl/ignite_environment.cpp    | 260 +++++++++++++-
 .../platforms/cpp/core/src/impl/ignite_impl.cpp |  15 +-
 .../core/src/impl/interop/interop_target.cpp    |  20 ++
 .../src/impl/transactions/transactions_impl.cpp |  10 -
 .../platforms/cpp/jni/include/ignite/jni/java.h |   1 +
 modules/platforms/cpp/jni/src/java.cpp          |  10 +
 58 files changed, 2493 insertions(+), 291 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/f9c96de5/modules/platforms/cpp/binary/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/binary/Makefile.am b/modules/platforms/cpp/binary/Makefile.am
index d3c2072..499c8a9 100644
--- a/modules/platforms/cpp/binary/Makefile.am
+++ b/modules/platforms/cpp/binary/Makefile.am
@@ -20,51 +20,52 @@ ACLOCAL_AMFLAGS =-I m4
 lib_LTLIBRARIES = libignite-binary.la
 
 SUBDIRS = \
-    include
+	include
 
 AM_CPPFLAGS = \
-    -I$(srcdir)/include \
-    -I@top_srcdir@/common/include \
-    -I@top_srcdir@/common/os/linux/include \
-    -DIGNITE_IMPL \
-    -D__STDC_LIMIT_MACROS \
-    -D__STDC_CONSTANT_MACROS
+	-I$(srcdir)/include \
+	-I@top_srcdir@/common/include \
+	-I@top_srcdir@/common/os/linux/include \
+	-DIGNITE_IMPL \
+	-D__STDC_LIMIT_MACROS \
+	-D__STDC_CONSTANT_MACROS
 
 AM_CXXFLAGS = \
-    -Wall \
-    -std=c++03
+	-Wall \
+	-std=c++03
 
 libignite_binary_la_LIBADD = \
-    @top_srcdir@/common/libignite-common.la
+	@top_srcdir@/common/libignite-common.la
 
 libignite_binary_la_LDFLAGS = \
-    -no-undefined \
-    -version-info 0:0:0 \
-    -release $(PACKAGE_VERSION)
+	-no-undefined \
+	-version-info 0:0:0 \
+	-release $(PACKAGE_VERSION)
 
 libignite_binary_la_DEPENDENCIES = \
-    @top_srcdir@/common/libignite-common.la
+	@top_srcdir@/common/libignite-common.la
 
 libignite_binary_la_SOURCES = \
-    src/binary/binary_containers.cpp \
-    src/binary/binary_raw_writer.cpp \
-    src/binary/binary_writer.cpp \
-    src/binary/binary_reader.cpp \
-    src/binary/binary_type.cpp \
-    src/binary/binary_raw_reader.cpp \
-    src/impl/binary/binary_type_manager.cpp \
-    src/impl/binary/binary_utils.cpp \
-    src/impl/binary/binary_reader_impl.cpp \
-    src/impl/binary/binary_type_handler.cpp \
-    src/impl/binary/binary_writer_impl.cpp \
-    src/impl/binary/binary_schema.cpp \
-    src/impl/binary/binary_type_snapshot.cpp \
-    src/impl/binary/binary_object_header.cpp \
-    src/impl/binary/binary_object_impl.cpp \
-    src/impl/binary/binary_field_meta.cpp \
-    src/impl/interop/interop_memory.cpp \
-    src/impl/interop/interop_output_stream.cpp \
-    src/impl/interop/interop_input_stream.cpp
+	src/binary/binary_containers.cpp \
+	src/binary/binary_raw_writer.cpp \
+	src/binary/binary_writer.cpp \
+	src/binary/binary_reader.cpp \
+	src/binary/binary_type.cpp \
+	src/binary/binary_raw_reader.cpp \
+	src/impl/binary/binary_type_manager.cpp \
+	src/impl/binary/binary_type_impl.cpp \
+	src/impl/binary/binary_utils.cpp \
+	src/impl/binary/binary_reader_impl.cpp \
+	src/impl/binary/binary_type_handler.cpp \
+	src/impl/binary/binary_writer_impl.cpp \
+	src/impl/binary/binary_schema.cpp \
+	src/impl/binary/binary_type_snapshot.cpp \
+	src/impl/binary/binary_object_header.cpp \
+	src/impl/binary/binary_object_impl.cpp \
+	src/impl/binary/binary_field_meta.cpp \
+	src/impl/interop/interop_memory.cpp \
+	src/impl/interop/interop_output_stream.cpp \
+	src/impl/interop/interop_input_stream.cpp
 
 clean-local:
 	$(RM) *.gcno *.gcda

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9c96de5/modules/platforms/cpp/binary/include/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/binary/include/Makefile.am b/modules/platforms/cpp/binary/include/Makefile.am
index a73d5de..85a5516 100644
--- a/modules/platforms/cpp/binary/include/Makefile.am
+++ b/modules/platforms/cpp/binary/include/Makefile.am
@@ -18,33 +18,34 @@
 ACLOCAL_AMFLAGS =-I m4
 
 nobase_include_HEADERS = \
-    ignite/binary/binary_raw_reader.h \
-    ignite/binary/binary_reader.h \
-    ignite/binary/binary_writer.h \
-    ignite/binary/binary_raw_writer.h \
-    ignite/binary/binary_containers.h \
-    ignite/binary/binary.h \
-    ignite/binary/binary_consts.h \
-    ignite/binary/binary_type.h \
-    ignite/binary/binary_object.h \
-    ignite/impl/binary/binary_type_handler.h \
-    ignite/impl/binary/binary_id_resolver.h \
-    ignite/impl/binary/binary_type_manager.h \
-    ignite/impl/binary/binary_type_updater.h \
-    ignite/impl/binary/binary_common.h \
-    ignite/impl/binary/binary_writer_impl.h \
-    ignite/impl/binary/binary_field_meta.h \
-    ignite/impl/binary/binary_type_snapshot.h \
-    ignite/impl/binary/binary_reader_impl.h \
-    ignite/impl/binary/binary_schema.h \
-    ignite/impl/binary/binary_utils.h \
-    ignite/impl/binary/binary_object_header.h \
-    ignite/impl/binary/binary_object_impl.h \
-    ignite/impl/interop/interop_memory.h \
-    ignite/impl/interop/interop.h \
-    ignite/impl/interop/interop_stream_position_guard.h \
-    ignite/impl/interop/interop_output_stream.h \
-    ignite/impl/interop/interop_input_stream.h
+	ignite/binary/binary_raw_reader.h \
+	ignite/binary/binary_reader.h \
+	ignite/binary/binary_writer.h \
+	ignite/binary/binary_raw_writer.h \
+	ignite/binary/binary_containers.h \
+	ignite/binary/binary.h \
+	ignite/binary/binary_consts.h \
+	ignite/binary/binary_type.h \
+	ignite/binary/binary_object.h \
+	ignite/impl/binary/binary_type_handler.h \
+	ignite/impl/binary/binary_id_resolver.h \
+	ignite/impl/binary/binary_type_impl.h \
+	ignite/impl/binary/binary_type_manager.h \
+	ignite/impl/binary/binary_type_updater.h \
+	ignite/impl/binary/binary_common.h \
+	ignite/impl/binary/binary_writer_impl.h \
+	ignite/impl/binary/binary_field_meta.h \
+	ignite/impl/binary/binary_type_snapshot.h \
+	ignite/impl/binary/binary_reader_impl.h \
+	ignite/impl/binary/binary_schema.h \
+	ignite/impl/binary/binary_utils.h \
+	ignite/impl/binary/binary_object_header.h \
+	ignite/impl/binary/binary_object_impl.h \
+	ignite/impl/interop/interop_memory.h \
+	ignite/impl/interop/interop.h \
+	ignite/impl/interop/interop_stream_position_guard.h \
+	ignite/impl/interop/interop_output_stream.h \
+	ignite/impl/interop/interop_input_stream.h
 
 uninstall-hook:
 	if [ -d ${includedir}/ignite ]; then find ${includedir}/ignite -type d -empty -delete; fi

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9c96de5/modules/platforms/cpp/binary/include/ignite/binary/binary_type.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/binary/include/ignite/binary/binary_type.h b/modules/platforms/cpp/binary/include/ignite/binary/binary_type.h
index 8fb9dc2..749404a 100644
--- a/modules/platforms/cpp/binary/include/ignite/binary/binary_type.h
+++ b/modules/platforms/cpp/binary/include/ignite/binary/binary_type.h
@@ -28,7 +28,7 @@
 
 #include <ignite/common/common.h>
 
-#include <ignite/ignite_error.h>
+#include <ignite/impl/binary/binary_type_impl.h>
 
 /**
  * @def IGNITE_BINARY_TYPE_START(T)

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9c96de5/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_type_impl.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_type_impl.h b/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_type_impl.h
new file mode 100644
index 0000000..2548a83
--- /dev/null
+++ b/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_type_impl.h
@@ -0,0 +1,61 @@
+/*
+ * 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.
+ */
+
+#ifndef _IGNITE_IMPL_BINARY_BINARY_TYPE_IMPL
+#define _IGNITE_IMPL_BINARY_BINARY_TYPE_IMPL
+
+#include <stdint.h>
+
+#include <ignite/ignite_error.h>
+
+namespace ignite
+{
+    namespace binary
+    {
+        class BinaryReader;
+        class BinaryWriter;
+
+        template<typename T>
+        struct BinaryType;
+
+        template<>
+        struct BinaryType<IgniteError>
+        {
+            static int32_t GetTypeId();
+
+            static void GetTypeName(std::string& dst)
+            {
+                dst = "IgniteError";
+            }
+
+            static int32_t GetFieldId(const char* name);
+
+            static bool IsNull(const IgniteError& obj)
+            {
+                return false;
+            }
+
+            static void GetNull(IgniteError& dst);
+
+            static void Write(BinaryWriter& writer, const IgniteError& obj);
+
+            static void Read(BinaryReader& reader, IgniteError& dst);
+        };
+    }
+}
+
+#endif //_IGNITE_IMPL_BINARY_BINARY_TYPE_IMPL

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9c96de5/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_writer_impl.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_writer_impl.h b/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_writer_impl.h
index d91e2f7..32801ec 100644
--- a/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_writer_impl.h
+++ b/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_writer_impl.h
@@ -59,7 +59,7 @@ namespace ignite
                  * @param metaMgr Type manager.
                  * @param metaHnd Type handler.
                  */
-                BinaryWriterImpl(ignite::impl::interop::InteropOutputStream* stream, BinaryIdResolver* idRslvr, 
+                BinaryWriterImpl(interop::InteropOutputStream* stream, BinaryIdResolver* idRslvr, 
                     BinaryTypeManager* metaMgr, BinaryTypeHandler* metaHnd, int32_t start);
                 
                 /**
@@ -69,7 +69,7 @@ namespace ignite
                  * @param stream Interop stream.
                  * @param metaMgr Type manager.
                  */
-                BinaryWriterImpl(ignite::impl::interop::InteropOutputStream* stream, BinaryTypeManager* metaMgr);
+                BinaryWriterImpl(interop::InteropOutputStream* stream, BinaryTypeManager* metaMgr);
 
                 /**
                  * Write 8-byte signed integer. Maps to "byte" type in Java.

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9c96de5/modules/platforms/cpp/binary/project/vs/binary.vcxproj
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/binary/project/vs/binary.vcxproj b/modules/platforms/cpp/binary/project/vs/binary.vcxproj
index 21f1248..5b76f06 100644
--- a/modules/platforms/cpp/binary/project/vs/binary.vcxproj
+++ b/modules/platforms/cpp/binary/project/vs/binary.vcxproj
@@ -194,6 +194,7 @@
     <ClInclude Include="..\..\include\ignite\impl\binary\binary_reader_impl.h" />
     <ClInclude Include="..\..\include\ignite\impl\binary\binary_schema.h" />
     <ClInclude Include="..\..\include\ignite\impl\binary\binary_type_handler.h" />
+    <ClInclude Include="..\..\include\ignite\impl\binary\binary_type_impl.h" />
     <ClInclude Include="..\..\include\ignite\impl\binary\binary_type_manager.h" />
     <ClInclude Include="..\..\include\ignite\impl\binary\binary_type_snapshot.h" />
     <ClInclude Include="..\..\include\ignite\impl\binary\binary_type_updater.h" />
@@ -218,6 +219,7 @@
     <ClCompile Include="..\..\src\impl\binary\binary_reader_impl.cpp" />
     <ClCompile Include="..\..\src\impl\binary\binary_schema.cpp" />
     <ClCompile Include="..\..\src\impl\binary\binary_type_handler.cpp" />
+    <ClCompile Include="..\..\src\impl\binary\binary_type_impl.cpp" />
     <ClCompile Include="..\..\src\impl\binary\binary_type_manager.cpp" />
     <ClCompile Include="..\..\src\impl\binary\binary_type_snapshot.cpp" />
     <ClCompile Include="..\..\src\impl\binary\binary_utils.cpp" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9c96de5/modules/platforms/cpp/binary/project/vs/binary.vcxproj.filters
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/binary/project/vs/binary.vcxproj.filters b/modules/platforms/cpp/binary/project/vs/binary.vcxproj.filters
index 27e4ad8..573b5fa 100644
--- a/modules/platforms/cpp/binary/project/vs/binary.vcxproj.filters
+++ b/modules/platforms/cpp/binary/project/vs/binary.vcxproj.filters
@@ -100,6 +100,9 @@
     <ClInclude Include="..\..\include\ignite\impl\binary\binary_field_meta.h">
       <Filter>Code\impl\binary</Filter>
     </ClInclude>
+    <ClInclude Include="..\..\include\ignite\impl\binary\binary_type_impl.h">
+      <Filter>Code\impl\binary</Filter>
+    </ClInclude>
   </ItemGroup>
   <ItemGroup>
     <ClCompile Include="..\..\src\binary\binary_containers.cpp">
@@ -159,5 +162,8 @@
     <ClCompile Include="..\..\src\impl\binary\binary_field_meta.cpp">
       <Filter>Code\impl\binary</Filter>
     </ClCompile>
+    <ClCompile Include="..\..\src\impl\binary\binary_type_impl.cpp">
+      <Filter>Code\impl\binary</Filter>
+    </ClCompile>
   </ItemGroup>
 </Project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9c96de5/modules/platforms/cpp/binary/src/impl/binary/binary_type_impl.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/binary/src/impl/binary/binary_type_impl.cpp b/modules/platforms/cpp/binary/src/impl/binary/binary_type_impl.cpp
new file mode 100644
index 0000000..59a4dda
--- /dev/null
+++ b/modules/platforms/cpp/binary/src/impl/binary/binary_type_impl.cpp
@@ -0,0 +1,61 @@
+/*
+ * 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.
+ */
+
+#include <cstring>
+
+#include <ignite/binary/binary_reader.h>
+#include <ignite/binary/binary_writer.h>
+#include <ignite/impl/binary/binary_type_impl.h>
+
+namespace ignite
+{
+    namespace binary
+    {
+        int32_t BinaryType<IgniteError>::GetTypeId()
+        {
+            return GetBinaryStringHashCode("IgniteError");
+        }
+
+        int32_t BinaryType<IgniteError>::GetFieldId(const char* name)
+        {
+            return GetBinaryStringHashCode(name);
+        }
+
+        void BinaryType<IgniteError>::GetNull(IgniteError& dst)
+        {
+            dst = IgniteError(0, 0);
+        }
+
+        void BinaryType<IgniteError>::Write(BinaryWriter& writer, const IgniteError& obj)
+        {
+            BinaryRawWriter raw = writer.RawWriter();
+
+            raw.WriteInt32(obj.GetCode());
+            raw.WriteString(obj.GetText(), static_cast<int32_t>(strlen(obj.GetText())));
+        }
+
+        void BinaryType<IgniteError>::Read(BinaryReader& reader, IgniteError& dst)
+        {
+            BinaryRawReader raw = reader.RawReader();
+
+            int32_t code = raw.ReadInt32();
+            std::string msg = raw.ReadObject<std::string>();
+
+            dst = IgniteError(code, msg.c_str());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9c96de5/modules/platforms/cpp/common/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/Makefile.am b/modules/platforms/cpp/common/Makefile.am
index 8515d5b..374f417 100644
--- a/modules/platforms/cpp/common/Makefile.am
+++ b/modules/platforms/cpp/common/Makefile.am
@@ -20,41 +20,41 @@ ACLOCAL_AMFLAGS =-I m4
 lib_LTLIBRARIES = libignite-common.la
 
 SUBDIRS = \
-    include \
-    os/linux/include
+	include \
+	os/linux/include
 
 AM_CPPFLAGS = \
-    -I$(srcdir)/include \
-    -I$(srcdir)/os/linux/include \
-    -DIGNITE_IMPL \
-    -D__STDC_LIMIT_MACROS \
-    -D__STDC_CONSTANT_MACROS
+	-I$(srcdir)/include \
+	-I$(srcdir)/os/linux/include \
+	-DIGNITE_IMPL \
+	-D__STDC_LIMIT_MACROS \
+	-D__STDC_CONSTANT_MACROS
 
 AM_CXXFLAGS = \
-    -Wall \
-    -std=c++03
+	-Wall \
+	-std=c++03
 
 libignite_common_la_LDFLAGS = \
-    -no-undefined \
-    -L/usr/local/lib \
-    -ldl \
-    -version-info 0:0:0 \
-    -release $(PACKAGE_VERSION)
+	-no-undefined \
+	-L/usr/local/lib \
+	-ldl \
+	-version-info 0:0:0 \
+	-release $(PACKAGE_VERSION)
 
 libignite_common_la_SOURCES = \
-    os/linux/src/common/concurrent_os.cpp \
-    os/linux/src/common/platform_utils.cpp \
-    os/linux/src/common/dynamic_load_os.cpp \
-    src/common/big_integer.cpp \
-    src/common/concurrent.cpp \
-    src/common/decimal.cpp \
-    src/common/bits.cpp \
-    src/common/utils.cpp \
-    src/date.cpp \
-    src/ignite_error.cpp \
-    src/guid.cpp \
-    src/timestamp.cpp \
-    src/time.cpp
+	os/linux/src/common/concurrent_os.cpp \
+	os/linux/src/common/platform_utils.cpp \
+	os/linux/src/common/dynamic_load_os.cpp \
+	src/common/big_integer.cpp \
+	src/common/concurrent.cpp \
+	src/common/decimal.cpp \
+	src/common/bits.cpp \
+	src/common/utils.cpp \
+	src/date.cpp \
+	src/ignite_error.cpp \
+	src/guid.cpp \
+	src/timestamp.cpp \
+	src/time.cpp
 
 clean-local:
 	$(RM) *.gcno *.gcda

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9c96de5/modules/platforms/cpp/common/include/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/include/Makefile.am b/modules/platforms/cpp/common/include/Makefile.am
index 64f0c46..f115c94 100644
--- a/modules/platforms/cpp/common/include/Makefile.am
+++ b/modules/platforms/cpp/common/include/Makefile.am
@@ -20,6 +20,7 @@ ACLOCAL_AMFLAGS =-I m4
 nobase_include_HEADERS = \
 	ignite/common/big_integer.h \
 	ignite/common/bits.h \
+	ignite/common/cancelable.h \
 	ignite/common/concurrent.h \
 	ignite/common/decimal.h \
 	ignite/common/default_allocator.h \

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9c96de5/modules/platforms/cpp/common/include/ignite/common/cancelable.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/include/ignite/common/cancelable.h b/modules/platforms/cpp/common/include/ignite/common/cancelable.h
new file mode 100644
index 0000000..ab0c126
--- /dev/null
+++ b/modules/platforms/cpp/common/include/ignite/common/cancelable.h
@@ -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.
+ */
+
+/**
+ * @file
+ * Declares ignite::common::Cancelable class.
+ */
+
+#ifndef _IGNITE_COMMON_CANCELABLE
+#define _IGNITE_COMMON_CANCELABLE
+
+#include <ignite/common/common.h>
+
+namespace ignite
+{
+    namespace common
+    {
+        /**
+         * Cancelable.
+         */
+        class IGNITE_IMPORT_EXPORT Cancelable
+        {
+        public:
+            /**
+             * Default constructor.
+             */
+            Cancelable()
+            {
+                // No-op.
+            }
+
+            /**
+             * Destructor.
+             */
+            virtual ~Cancelable()
+            {
+                // No-op.
+            }
+
+            /**
+             * Cancels the operation.
+             */
+            virtual void Cancel() = 0;
+
+        private:
+            IGNITE_NO_COPY_ASSIGNMENT(Cancelable);
+        };
+    }
+}
+
+#endif //_IGNITE_COMMON_CANCELABLE

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9c96de5/modules/platforms/cpp/common/include/ignite/common/promise.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/include/ignite/common/promise.h b/modules/platforms/cpp/common/include/ignite/common/promise.h
index 548b76b..2e6152f 100644
--- a/modules/platforms/cpp/common/include/ignite/common/promise.h
+++ b/modules/platforms/cpp/common/include/ignite/common/promise.h
@@ -111,9 +111,18 @@ namespace ignite
                 state.Get()->SetError(err);
             }
 
+            /**
+             * Set cancel target.
+             */
+            void SetCancelTarget(std::auto_ptr<Cancelable>& target)
+            {
+                state.Get()->SetCancelTarget(target);
+            }
+
         private:
             IGNITE_NO_COPY_ASSIGNMENT(Promise);
 
+            /** Shared state. */
             concurrent::SharedPointer< SharedState<ValueType> > state;
         };
 
@@ -190,9 +199,18 @@ namespace ignite
                 state.Get()->SetError(err);
             }
 
+            /**
+             * Set cancel target.
+             */
+            void SetCancelTarget(std::auto_ptr<Cancelable>& target)
+            {
+                state.Get()->SetCancelTarget(target);
+            }
+
         private:
             IGNITE_NO_COPY_ASSIGNMENT(Promise);
 
+            /** Shared state. */
             concurrent::SharedPointer< SharedState<ValueType> > state;
         };
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9c96de5/modules/platforms/cpp/common/include/ignite/common/shared_state.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/include/ignite/common/shared_state.h b/modules/platforms/cpp/common/include/ignite/common/shared_state.h
index 8886532..d223753 100644
--- a/modules/platforms/cpp/common/include/ignite/common/shared_state.h
+++ b/modules/platforms/cpp/common/include/ignite/common/shared_state.h
@@ -23,8 +23,11 @@
 #ifndef _IGNITE_SHARED_STATE
 #define _IGNITE_SHARED_STATE
 
+#include <memory>
+
 #include <ignite/common/common.h>
 #include <ignite/common/concurrent.h>
+#include <ignite/common/cancelable.h>
 #include <ignite/ignite_error.h>
 
 namespace ignite
@@ -162,9 +165,33 @@ namespace ignite
                 throw error;
             }
 
+            /**
+             * Set cancel target.
+             */
+            void SetCancelTarget(std::auto_ptr<Cancelable>& target)
+            {
+                concurrent::CsLockGuard guard(mutex);
+
+                cancelTarget = target;
+            }
+
+            /**
+             * Cancel related operation.
+             */
+            void Cancel()
+            {
+                concurrent::CsLockGuard guard(mutex);
+
+                if (cancelTarget.get())
+                    cancelTarget->Cancel();
+            }
+
         private:
             IGNITE_NO_COPY_ASSIGNMENT(SharedState);
 
+            /** Cancel target. */
+            std::auto_ptr<Cancelable> cancelTarget;
+
             /** Value. */
             std::auto_ptr<ValueType> value;
 
@@ -310,9 +337,33 @@ namespace ignite
                 throw error;
             }
 
+            /**
+             * Set cancel target.
+             */
+            void SetCancelTarget(std::auto_ptr<Cancelable>& target)
+            {
+                concurrent::CsLockGuard guard(mutex);
+
+                cancelTarget = target;
+            }
+
+            /**
+             * Cancel related operation.
+             */
+            void Cancel()
+            {
+                concurrent::CsLockGuard guard(mutex);
+
+                if (cancelTarget.get())
+                    cancelTarget->Cancel();
+            }
+
         private:
             IGNITE_NO_COPY_ASSIGNMENT(SharedState);
 
+            /** Cancel target. */
+            std::auto_ptr<Cancelable> cancelTarget;
+
             /** Marker. */
             bool done;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9c96de5/modules/platforms/cpp/common/include/ignite/future.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/include/ignite/future.h b/modules/platforms/cpp/common/include/ignite/future.h
index 5c42e55..f709797 100644
--- a/modules/platforms/cpp/common/include/ignite/future.h
+++ b/modules/platforms/cpp/common/include/ignite/future.h
@@ -85,7 +85,7 @@ namespace ignite
 
             assert(state0 != 0);
 
-            state.Get()->Wait();
+            state0->Wait();
         }
 
         /**
@@ -101,7 +101,7 @@ namespace ignite
 
             assert(state0 != 0);
 
-            return state.Get()->WaitFor(msTimeout);
+            return state0->WaitFor(msTimeout);
         }
 
         /**
@@ -117,7 +117,31 @@ namespace ignite
 
             assert(state0 != 0);
 
-            return state.Get()->GetValue();
+            return state0->GetValue();
+        }
+
+        /**
+         * Cancel related operation.
+         */
+        void Cancel()
+        {
+            common::SharedState<ValueType>* state0 = state.Get();
+
+            assert(state0 != 0);
+
+            state0->Cancel();
+        }
+
+        /**
+         * Check if the future ready.
+         */
+        bool IsReady()
+        {
+            common::SharedState<ValueType>* state0 = state.Get();
+
+            assert(state0 != 0);
+
+            return state0->IsSet();
         }
 
     private:
@@ -182,7 +206,7 @@ namespace ignite
 
             assert(state0 != 0);
 
-            state.Get()->Wait();
+            state0->Wait();
         }
 
         /**
@@ -198,7 +222,7 @@ namespace ignite
 
             assert(state0 != 0);
 
-            return state.Get()->WaitFor(msTimeout);
+            return state0->WaitFor(msTimeout);
         }
 
         /**
@@ -213,7 +237,31 @@ namespace ignite
 
             assert(state0 != 0);
 
-            state.Get()->GetValue();
+            state0->GetValue();
+        }
+
+        /**
+         * Cancel related operation.
+         */
+        void Cancel()
+        {
+            common::SharedState<ValueType>* state0 = state.Get();
+
+            assert(state0 != 0);
+
+            state0->Cancel();
+        }
+
+        /**
+         * Check if the future ready.
+         */
+        bool IsReady()
+        {
+            common::SharedState<ValueType>* state0 = state.Get();
+
+            assert(state0 != 0);
+
+            return state0->IsSet();
         }
 
     private:

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9c96de5/modules/platforms/cpp/common/include/ignite/ignite_error.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/include/ignite/ignite_error.h b/modules/platforms/cpp/common/include/ignite/ignite_error.h
index 7818dd2..59bb0c4 100644
--- a/modules/platforms/cpp/common/include/ignite/ignite_error.h
+++ b/modules/platforms/cpp/common/include/ignite/ignite_error.h
@@ -124,6 +124,9 @@ namespace ignite
         /** Binary error. */
         static const int IGNITE_ERR_BINARY = 1002;
 
+        /** Standard library exception. */
+        static const int IGNITE_ERR_STD = 1003;
+
         /** Generic %Ignite error. */
         static const int IGNITE_ERR_GENERIC = 2000;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9c96de5/modules/platforms/cpp/common/project/vs/common.vcxproj
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/project/vs/common.vcxproj b/modules/platforms/cpp/common/project/vs/common.vcxproj
index 9f32461..07c3f50 100644
--- a/modules/platforms/cpp/common/project/vs/common.vcxproj
+++ b/modules/platforms/cpp/common/project/vs/common.vcxproj
@@ -166,6 +166,7 @@
   </ItemDefinitionGroup>
   <ItemGroup>
     <ClInclude Include="..\..\include\ignite\common\big_integer.h" />
+    <ClInclude Include="..\..\include\ignite\common\cancelable.h" />
     <ClInclude Include="..\..\include\ignite\common\concurrent.h" />
     <ClInclude Include="..\..\include\ignite\common\decimal.h" />
     <ClInclude Include="..\..\include\ignite\common\default_allocator.h" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9c96de5/modules/platforms/cpp/common/project/vs/common.vcxproj.filters
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/project/vs/common.vcxproj.filters b/modules/platforms/cpp/common/project/vs/common.vcxproj.filters
index 4daf3aa..020d32b 100644
--- a/modules/platforms/cpp/common/project/vs/common.vcxproj.filters
+++ b/modules/platforms/cpp/common/project/vs/common.vcxproj.filters
@@ -76,6 +76,9 @@
     <ClInclude Include="..\..\include\ignite\common\shared_state.h">
       <Filter>Code\common</Filter>
     </ClInclude>
+    <ClInclude Include="..\..\include\ignite\common\cancelable.h">
+      <Filter>Code\common</Filter>
+    </ClInclude>
   </ItemGroup>
   <ItemGroup>
     <ClCompile Include="..\..\src\date.cpp">

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9c96de5/modules/platforms/cpp/core-test/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/Makefile.am b/modules/platforms/cpp/core-test/Makefile.am
index 11f4d1a..f87effd 100644
--- a/modules/platforms/cpp/core-test/Makefile.am
+++ b/modules/platforms/cpp/core-test/Makefile.am
@@ -20,67 +20,68 @@ ACLOCAL_AMFLAGS =-I m4
 noinst_PROGRAMS = ignite-tests
 
 SUBDIRS = \
-    include
+	include
 
 AM_CPPFLAGS = \
-    -I$(srcdir)/include \
-    -I@top_srcdir@/core/include \
-    -I@top_srcdir@/core/os/linux/include \
-    -I@top_srcdir@/common/include \
-    -I@top_srcdir@/common/os/linux/include \
-    -I@top_srcdir@/binary/include \
-    -I@top_srcdir@/jni/include \
-    -I@top_srcdir@/jni/os/linux/include \
-    -I$(JAVA_HOME)/include \
-    -I$(JAVA_HOME)/include/linux \
-    -DIGNITE_IMPL \
-    -D__STDC_LIMIT_MACROS \
-    -D__STDC_CONSTANT_MACROS
+	-I$(srcdir)/include \
+	-I@top_srcdir@/core/include \
+	-I@top_srcdir@/core/os/linux/include \
+	-I@top_srcdir@/common/include \
+	-I@top_srcdir@/common/os/linux/include \
+	-I@top_srcdir@/binary/include \
+	-I@top_srcdir@/jni/include \
+	-I@top_srcdir@/jni/os/linux/include \
+	-I$(JAVA_HOME)/include \
+	-I$(JAVA_HOME)/include/linux \
+	-DIGNITE_IMPL \
+	-D__STDC_LIMIT_MACROS \
+	-D__STDC_CONSTANT_MACROS
 
 AM_CXXFLAGS = \
-    -Wall \
-    -std=c++03
+	-Wall \
+	-std=c++03
 
 ignite_tests_LDADD = \
-    @top_srcdir@/core/libignite.la \
-    -lpthread \
-    -lboost_thread \
-    -lboost_system \
-    -lboost_chrono
+	@top_srcdir@/core/libignite.la \
+	-lpthread \
+	-lboost_thread \
+	-lboost_system \
+	-lboost_chrono
 
 ignite_tests_LDFLAGS = \
-    -static-libtool-libs \
-    -rdynamic
+	-static-libtool-libs \
+	-rdynamic
 
 ignite_tests_SOURCES = \
-    src/reference_test.cpp \
-    src/bits_test.cpp \
+	src/reference_test.cpp \
+	src/bits_test.cpp \
 	src/binary_identity_resolver_test.cpp \
-    src/cache_test.cpp \
-    src/cache_query_test.cpp \
-    src/cache_store_test.cpp \
-    src/continuous_query_test.cpp \
-    src/concurrent_test.cpp \
-    src/ignition_test.cpp \
-    src/interop_memory_test.cpp \
-    src/interop_test.cpp \
-    src/cluster_test.cpp \
-    src/cache_invoke_test.cpp \
-    src/handle_registry_test.cpp \
-    src/ignite_error_test.cpp \
-    src/binary_test_defs.cpp \
-    src/binary_object_test.cpp \
-    src/binary_reader_writer_raw_test.cpp \
-    src/binary_reader_writer_test.cpp \
-    src/binary_session_test.cpp \
-    src/date_time_test.cpp \
-    src/decimal_test.cpp \
-    src/dynamic_size_array_test.cpp \
-    src/fixed_size_array_test.cpp \
-    src/future_test.cpp \
-    src/transactions_test.cpp \
-    src/teamcity_messages.cpp \
-    src/teamcity_boost.cpp \
+	src/cache_test.cpp \
+	src/cache_query_test.cpp \
+	src/cache_store_test.cpp \
+	src/continuous_query_test.cpp \
+	src/concurrent_test.cpp \
+	src/compute_test.cpp \
+	src/ignition_test.cpp \
+	src/interop_memory_test.cpp \
+	src/interop_test.cpp \
+	src/cluster_test.cpp \
+	src/cache_invoke_test.cpp \
+	src/handle_registry_test.cpp \
+	src/ignite_error_test.cpp \
+	src/binary_test_defs.cpp \
+	src/binary_object_test.cpp \
+	src/binary_reader_writer_raw_test.cpp \
+	src/binary_reader_writer_test.cpp \
+	src/binary_session_test.cpp \
+	src/date_time_test.cpp \
+	src/decimal_test.cpp \
+	src/dynamic_size_array_test.cpp \
+	src/fixed_size_array_test.cpp \
+	src/future_test.cpp \
+	src/transactions_test.cpp \
+	src/teamcity_messages.cpp \
+	src/teamcity_boost.cpp \
 	src/test_utils.cpp
 
 run-check: check

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9c96de5/modules/platforms/cpp/core-test/include/ignite/test_utils.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/include/ignite/test_utils.h b/modules/platforms/cpp/core-test/include/ignite/test_utils.h
index 67fda95..aa91458 100644
--- a/modules/platforms/cpp/core-test/include/ignite/test_utils.h
+++ b/modules/platforms/cpp/core-test/include/ignite/test_utils.h
@@ -22,6 +22,11 @@
 
 namespace ignite_test
 {
+    enum
+    {
+        TEST_ERROR = 424242
+    };
+
     /**
      * Initialize configuration for a node.
      *
@@ -65,9 +70,24 @@ namespace ignite_test
      * @param err Error.
      * @return True if the error is generic.
      */
-    inline bool IsGenericError(const ignite::IgniteError& err)
+    bool IsGenericError(const ignite::IgniteError& err);
+
+    /**
+     * Check if the error is generic.
+     *
+     * @param err Error.
+     * @return True if the error is generic.
+     */
+    bool IsTestError(const ignite::IgniteError& err);
+
+    /**
+     * Make test error.
+     *
+     * @return Test error.
+     */
+    inline ignite::IgniteError MakeTestError()
     {
-        return err.GetCode() == ignite::IgniteError::IGNITE_ERR_GENERIC;
+        return ignite::IgniteError(TEST_ERROR, "Test error");
     }
 }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9c96de5/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj b/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj
index 51ae5d41..f2f6a1b 100644
--- a/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj
+++ b/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj
@@ -64,6 +64,7 @@
     <ClCompile Include="..\..\src\cache_test.cpp" />
     <ClCompile Include="..\..\src\cluster_test.cpp" />
     <ClCompile Include="..\..\src\cache_invoke_test.cpp" />
+    <ClCompile Include="..\..\src\compute_test.cpp" />
     <ClCompile Include="..\..\src\concurrent_test.cpp" />
     <ClCompile Include="..\..\src\date_time_test.cpp" />
     <ClCompile Include="..\..\src\decimal_test.cpp" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9c96de5/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj.filters
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj.filters b/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj.filters
index ebccc7f..d58ef97 100644
--- a/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj.filters
+++ b/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj.filters
@@ -88,6 +88,9 @@
     <ClCompile Include="..\..\src\future_test.cpp">
       <Filter>Code</Filter>
     </ClCompile>
+    <ClCompile Include="..\..\src\compute_test.cpp">
+      <Filter>Code</Filter>
+    </ClCompile>
   </ItemGroup>
   <ItemGroup>
     <ClInclude Include="..\..\include\teamcity_messages.h">
@@ -166,9 +169,6 @@
     <None Include="..\..\config\cache-identity-default.xml">
       <Filter>Configs</Filter>
     </None>
-    <None Include="..\..\config\cache-query-continuous.xml">
-      <Filter>Configs</Filter>
-    </None>
     <None Include="..\..\config\cache-store-default.xml">
       <Filter>Configs</Filter>
     </None>
@@ -178,9 +178,5 @@
     <None Include="..\..\config\cache-store-32.xml">
       <Filter>Configs</Filter>
     </None>
-    <None Include="..\..\config\cache-query-continuous.xml">
-      <Filter>Configs</Filter>
-    </None>
-    <None Include="..\..\config\cache-query-continuous.xml" />
   </ItemGroup>
 </Project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9c96de5/modules/platforms/cpp/core-test/src/cache_invoke_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/src/cache_invoke_test.cpp b/modules/platforms/cpp/core-test/src/cache_invoke_test.cpp
index 0c30af8..55fca85 100644
--- a/modules/platforms/cpp/core-test/src/cache_invoke_test.cpp
+++ b/modules/platforms/cpp/core-test/src/cache_invoke_test.cpp
@@ -379,7 +379,7 @@ namespace ignite
 
 IGNITE_EXPORTED_CALL void IgniteModuleInit(ignite::IgniteBindingContext& context)
 {
-    IgniteBinding binding = context.GetBingding();
+    IgniteBinding binding = context.GetBinding();
 
     binding.RegisterCacheEntryProcessor<CacheEntryModifier>();
     binding.RegisterCacheEntryProcessor<Divisor>();

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9c96de5/modules/platforms/cpp/core-test/src/cluster_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/src/cluster_test.cpp b/modules/platforms/cpp/core-test/src/cluster_test.cpp
index 754fe7e..e9d6728 100644
--- a/modules/platforms/cpp/core-test/src/cluster_test.cpp
+++ b/modules/platforms/cpp/core-test/src/cluster_test.cpp
@@ -80,7 +80,7 @@ BOOST_AUTO_TEST_CASE(IgniteImplForServers)
 
     IgniteError err;
 
-    BOOST_REQUIRE(clusterGroup.Get()->ForServers(err).IsValid());
+    BOOST_REQUIRE(clusterGroup.Get()->ForServers().IsValid());
 }
 
 BOOST_AUTO_TEST_SUITE_END()
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9c96de5/modules/platforms/cpp/core-test/src/compute_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/src/compute_test.cpp b/modules/platforms/cpp/core-test/src/compute_test.cpp
new file mode 100644
index 0000000..d3b1183
--- /dev/null
+++ b/modules/platforms/cpp/core-test/src/compute_test.cpp
@@ -0,0 +1,337 @@
+/*
+ * 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.
+ */
+
+#ifndef _MSC_VER
+    #define BOOST_TEST_DYN_LINK
+#endif
+
+#include <boost/test/unit_test.hpp>
+#include <boost/chrono.hpp>
+#include <boost/thread.hpp>
+
+#include <ignite/ignition.h>
+#include <ignite/test_utils.h>
+
+#include <ignite/test_utils.h>
+
+using namespace ignite;
+using namespace ignite::compute;
+using namespace ignite::common::concurrent;
+using namespace ignite_test;
+
+using namespace boost::unit_test;
+
+/*
+ * Test setup fixture.
+ */
+struct ComputeTestSuiteFixture
+{
+    Ignite node;
+
+    Ignite MakeNode(const char* name)
+    {
+#ifdef IGNITE_TESTS_32
+        const char* config = "cache-test-32.xml";
+#else
+        const char* config = "cache-test.xml";
+#endif
+        return StartNode(config, name);
+    }
+
+    /*
+     * Constructor.
+     */
+    ComputeTestSuiteFixture() :
+        node(MakeNode("ComputeNode1"))
+    {
+        // No-op.
+    }
+
+    /*
+     * Destructor.
+     */
+    ~ComputeTestSuiteFixture()
+    {
+        Ignition::StopAll(true);
+    }
+};
+
+struct Func1 : ComputeFunc<std::string>
+{
+    Func1() :
+        a(), b(), err()
+    {
+        // No-op.
+    }
+
+    Func1(int32_t a, int32_t b) :
+        a(a), b(b), err()
+    {
+        // No-op.
+    }
+
+    Func1(IgniteError err) :
+        a(), b(), err(err)
+    {
+        // No-op.
+    }
+
+    virtual std::string Call()
+    {
+        if (err.GetCode() != IgniteError::IGNITE_SUCCESS)
+            throw err;
+
+        std::stringstream tmp;
+
+        tmp << a << '.' << b;
+
+        return tmp.str();
+    }
+
+    int32_t a;
+    int32_t b;
+    IgniteError err;
+};
+
+struct Func2 : ComputeFunc<std::string>
+{
+    Func2() :
+        a(), b(), err()
+    {
+        // No-op.
+    }
+
+    Func2(int32_t a, int32_t b) :
+        a(a), b(b), err()
+    {
+        // No-op.
+    }
+
+    Func2(IgniteError err) :
+        a(), b(), err(err)
+    {
+        // No-op.
+    }
+
+    virtual std::string Call()
+    {
+        boost::this_thread::sleep_for(boost::chrono::milliseconds(200));
+
+        if (err.GetCode() != IgniteError::IGNITE_SUCCESS)
+            throw err;
+
+        std::stringstream tmp;
+
+        tmp << a << '.' << b;
+
+        return tmp.str();
+    }
+
+    int32_t a;
+    int32_t b;
+    IgniteError err;
+};
+
+namespace ignite
+{
+    namespace binary
+    {
+        template<>
+        struct BinaryType<Func1>
+        {
+            static int32_t GetTypeId()
+            {
+                return GetBinaryStringHashCode("Func1");
+            }
+
+            static void GetTypeName(std::string& dst)
+            {
+                dst = "Func1";
+            }
+
+            static int32_t GetFieldId(const char* name)
+            {
+                return GetBinaryStringHashCode(name);
+            }
+
+            static bool IsNull(const Func1& obj)
+            {
+                return false;
+            }
+
+            static void GetNull(Func1& dst)
+            {
+                dst = Func1(0, 0);
+            }
+
+            static void Write(BinaryWriter& writer, const Func1& obj)
+            {
+                writer.WriteInt32("a", obj.a);
+                writer.WriteInt32("b", obj.b);
+                writer.WriteObject<IgniteError>("err", obj.err);
+            }
+
+            static void Read(BinaryReader& reader, Func1& dst)
+            {
+                dst.a = reader.ReadInt32("a");
+                dst.b = reader.ReadInt32("b");
+                dst.err = reader.ReadObject<IgniteError>("err");
+            }
+        };
+
+        template<>
+        struct BinaryType<Func2>
+        {
+            static int32_t GetTypeId()
+            {
+                return GetBinaryStringHashCode("Func2");
+            }
+
+            static void GetTypeName(std::string& dst)
+            {
+                dst = "Func2";
+            }
+
+            static int32_t GetFieldId(const char* name)
+            {
+                return GetBinaryStringHashCode(name);
+            }
+
+            static bool IsNull(const Func2& obj)
+            {
+                return false;
+            }
+
+            static void GetNull(Func2& dst)
+            {
+                dst = Func2(0, 0);
+            }
+
+            static void Write(BinaryWriter& writer, const Func2& obj)
+            {
+                writer.WriteInt32("a", obj.a);
+                writer.WriteInt32("b", obj.b);
+                writer.WriteObject<IgniteError>("err", obj.err);
+            }
+
+            static void Read(BinaryReader& reader, Func2& dst)
+            {
+                dst.a = reader.ReadInt32("a");
+                dst.b = reader.ReadInt32("b");
+                dst.err = reader.ReadObject<IgniteError>("err");
+            }
+        };
+    }
+}
+
+IGNITE_EXPORTED_CALL void IgniteModuleInit1(IgniteBindingContext& context)
+{
+    IgniteBinding binding = context.GetBinding();
+
+    binding.RegisterComputeFunc<Func1>();
+    binding.RegisterComputeFunc<Func2>();
+}
+
+BOOST_FIXTURE_TEST_SUITE(ComputeTestSuite, ComputeTestSuiteFixture)
+
+BOOST_AUTO_TEST_CASE(IgniteCallSyncLocal)
+{
+    Compute compute = node.GetCompute();
+
+    BOOST_CHECKPOINT("Making Call");
+    std::string res = compute.Call<std::string>(Func1(8, 5));
+
+    BOOST_CHECK_EQUAL(res, "8.5");
+}
+
+BOOST_AUTO_TEST_CASE(IgniteCallAsyncLocal)
+{
+    Compute compute = node.GetCompute();
+
+    BOOST_CHECKPOINT("Making Call");
+    Future<std::string> res = compute.CallAsync<std::string>(Func2(312, 245));
+
+    BOOST_CHECK(!res.IsReady());
+
+    BOOST_CHECKPOINT("Waiting with timeout");
+    res.WaitFor(100);
+
+    BOOST_CHECK(!res.IsReady());
+
+    BOOST_CHECK_EQUAL(res.GetValue(), "312.245");
+}
+
+BOOST_AUTO_TEST_CASE(IgniteCallSyncLocalError)
+{
+    Compute compute = node.GetCompute();
+
+    BOOST_CHECKPOINT("Making Call");
+
+    BOOST_CHECK_EXCEPTION(compute.Call<std::string>(Func1(MakeTestError())), IgniteError, IsTestError);
+}
+
+BOOST_AUTO_TEST_CASE(IgniteCallAsyncLocalError)
+{
+    Compute compute = node.GetCompute();
+
+    BOOST_CHECKPOINT("Making Call");
+    Future<std::string> res = compute.CallAsync<std::string>(Func2(MakeTestError()));
+
+    BOOST_CHECK(!res.IsReady());
+
+    BOOST_CHECKPOINT("Waiting with timeout");
+    res.WaitFor(100);
+
+    BOOST_CHECK(!res.IsReady());
+
+    BOOST_CHECK_EXCEPTION(res.GetValue(), IgniteError, IsTestError);
+}
+
+BOOST_AUTO_TEST_CASE(IgniteCallTestRemote)
+{
+    Ignite node2 = MakeNode("ComputeNode2");
+    Compute compute = node.GetCompute();
+
+    BOOST_CHECKPOINT("Making Call");
+    compute.CallAsync<std::string>(Func2(8, 5));
+
+    std::string res = compute.Call<std::string>(Func1(42, 24));
+
+    BOOST_CHECK_EQUAL(res, "42.24");
+}
+
+BOOST_AUTO_TEST_CASE(IgniteCallTestRemoteError)
+{
+    Ignite node2 = MakeNode("ComputeNode2");
+    Compute compute = node.GetCompute();
+
+    BOOST_CHECKPOINT("Making Call");
+    compute.CallAsync<std::string>(Func2(8, 5));
+
+    Future<std::string> res = compute.CallAsync<std::string>(Func2(MakeTestError()));
+
+    BOOST_CHECK(!res.IsReady());
+
+    BOOST_CHECKPOINT("Waiting with timeout");
+    res.WaitFor(100);
+
+    BOOST_CHECK(!res.IsReady());
+
+    BOOST_CHECK_EXCEPTION(res.GetValue(), IgniteError, IsTestError);
+}
+
+BOOST_AUTO_TEST_SUITE_END()
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9c96de5/modules/platforms/cpp/core-test/src/continuous_query_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/src/continuous_query_test.cpp b/modules/platforms/cpp/core-test/src/continuous_query_test.cpp
index e092728..b869eb6 100644
--- a/modules/platforms/cpp/core-test/src/continuous_query_test.cpp
+++ b/modules/platforms/cpp/core-test/src/continuous_query_test.cpp
@@ -371,7 +371,7 @@ void CheckEvents(Cache<int, TestEntry>& cache, Listener<int, TestEntry>& lsnr)
 
 IGNITE_EXPORTED_CALL void IgniteModuleInit0(ignite::IgniteBindingContext& context)
 {
-    IgniteBinding binding = context.GetBingding();
+    IgniteBinding binding = context.GetBinding();
 
     binding.RegisterCacheEntryEventFilter< RangeFilter<int, TestEntry> >();
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9c96de5/modules/platforms/cpp/core-test/src/test_utils.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/src/test_utils.cpp b/modules/platforms/cpp/core-test/src/test_utils.cpp
index 7323028..c6e4454 100644
--- a/modules/platforms/cpp/core-test/src/test_utils.cpp
+++ b/modules/platforms/cpp/core-test/src/test_utils.cpp
@@ -83,4 +83,13 @@ namespace ignite_test
         return Ignition::Start(cfg, name);
     }
 
+    bool IsGenericError(const ignite::IgniteError& err)
+    {
+        return err.GetCode() == ignite::IgniteError::IGNITE_ERR_GENERIC;
+    }
+
+    bool IsTestError(const ignite::IgniteError& err)
+    {
+        return err.GetCode() == TEST_ERROR;
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9c96de5/modules/platforms/cpp/core/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/Makefile.am b/modules/platforms/cpp/core/Makefile.am
index 4de45d3..1c970e8 100644
--- a/modules/platforms/cpp/core/Makefile.am
+++ b/modules/platforms/cpp/core/Makefile.am
@@ -20,58 +20,60 @@ ACLOCAL_AMFLAGS =-I m4
 lib_LTLIBRARIES = libignite.la
 
 SUBDIRS = \
-    include
+	include
 
 AM_CPPFLAGS = \
-    -I$(srcdir)/include \
-    -I@top_srcdir@/common/include \
-    -I@top_srcdir@/common/os/linux/include \
-    -I@top_srcdir@/binary/include \
-    -I@top_srcdir@/jni/include \
-    -I@top_srcdir@/jni/os/linux/include \
-    -I$(JAVA_HOME)/include \
-    -I$(JAVA_HOME)/include/linux \
-    -DIGNITE_IMPL \
-    -D__STDC_LIMIT_MACROS \
-    -D__STDC_CONSTANT_MACROS
+	-I$(srcdir)/include \
+	-I@top_srcdir@/common/include \
+	-I@top_srcdir@/common/os/linux/include \
+	-I@top_srcdir@/binary/include \
+	-I@top_srcdir@/jni/include \
+	-I@top_srcdir@/jni/os/linux/include \
+	-I$(JAVA_HOME)/include \
+	-I$(JAVA_HOME)/include/linux \
+	-DIGNITE_IMPL \
+	-D__STDC_LIMIT_MACROS \
+	-D__STDC_CONSTANT_MACROS
 
 AM_CXXFLAGS = \
-    -Wall \
-    -std=c++03
+	-Wall \
+	-std=c++03
 
 libignite_la_LIBADD = \
-    @top_srcdir@/jni/libignite-jni.la \
-    @top_srcdir@/binary/libignite-binary.la
+	@top_srcdir@/jni/libignite-jni.la \
+	@top_srcdir@/binary/libignite-binary.la
 
 libignite_la_LDFLAGS = \
-    -no-undefined \
-    -ldl \
-    -version-info 0:0:0 \
-    -release $(PACKAGE_VERSION)
+	-no-undefined \
+	-ldl \
+	-version-info 0:0:0 \
+	-release $(PACKAGE_VERSION)
 
 libignite_la_DEPENDENCIES = \
-    @top_srcdir@/jni/libignite-jni.la \
-    @top_srcdir@/binary/libignite-binary.la
+	@top_srcdir@/jni/libignite-jni.la \
+	@top_srcdir@/binary/libignite-binary.la
 
 libignite_la_SOURCES = \
-    src/ignite.cpp \
-    src/ignition.cpp \
-    src/impl/ignite_environment.cpp \
-    src/impl/binary/binary_type_updater_impl.cpp \
-    src/impl/handle_registry.cpp \
-    src/impl/cache/query/continuous/continuous_query_handle_impl.cpp \
-    src/impl/cache/query/query_impl.cpp \
-    src/impl/cache/cache_impl.cpp \
-    src/impl/cache/query/query_batch.cpp \
-    src/impl/interop/interop_external_memory.cpp \
-    src/impl/interop/interop_target.cpp \
-    src/impl/transactions/transaction_impl.cpp \
-    src/impl/transactions/transactions_impl.cpp \
-    src/impl/cluster/cluster_group_impl.cpp \
-    src/impl/ignite_impl.cpp \
-    src/impl/ignite_binding_impl.cpp \
-    src/transactions/transaction.cpp \
-    src/transactions/transactions.cpp
+	src/ignite.cpp \
+	src/ignition.cpp \
+	src/impl/ignite_environment.cpp \
+	src/impl/binary/binary_type_updater_impl.cpp \
+	src/impl/handle_registry.cpp \
+	src/impl/cache/query/continuous/continuous_query_handle_impl.cpp \
+	src/impl/cache/query/query_impl.cpp \
+	src/impl/cache/cache_impl.cpp \
+	src/impl/cache/query/query_batch.cpp \
+	src/impl/interop/interop_external_memory.cpp \
+	src/impl/interop/interop_target.cpp \
+	src/impl/transactions/transaction_impl.cpp \
+	src/impl/transactions/transactions_impl.cpp \
+	src/impl/cluster/cluster_group_impl.cpp \
+	src/impl/compute/cancelable_impl.cpp \
+	src/impl/compute/compute_impl.cpp \
+	src/impl/ignite_impl.cpp \
+	src/impl/ignite_binding_impl.cpp \
+	src/transactions/transaction.cpp \
+	src/transactions/transactions.cpp
 
 pkgconfigdir = $(libdir)/pkgconfig
 pkgconfig_DATA = $(srcdir)/ignite.pc

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9c96de5/modules/platforms/cpp/core/include/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/Makefile.am b/modules/platforms/cpp/core/include/Makefile.am
index 0e9a7ec..50772cb 100644
--- a/modules/platforms/cpp/core/include/Makefile.am
+++ b/modules/platforms/cpp/core/include/Makefile.am
@@ -18,56 +18,63 @@
 ACLOCAL_AMFLAGS =-I m4
 
 nobase_include_HEADERS = \
-    ignite/cache/cache.h \
-    ignite/cache/cache_entry.h \
-    ignite/cache/cache_entry_processor.h \
-    ignite/cache/cache_peek_mode.h \
-    ignite/cache/event/cache_entry_event.h \
-    ignite/cache/event/cache_entry_event_filter.h \
-    ignite/cache/event/cache_entry_event_listener.h \
-    ignite/cache/mutable_cache_entry.h \
-    ignite/cache/query/continuous/continuous_query.h \
-    ignite/cache/query/continuous/continuous_query_handle.h \
-    ignite/cache/query/query.h \
-    ignite/cache/query/query_cursor.h \
-    ignite/cache/query/query_fields_cursor.h \
-    ignite/cache/query/query_fields_row.h \
-    ignite/cache/query/query_scan.h \
-    ignite/cache/query/query_sql.h \
-    ignite/cache/query/query_sql_fields.h \
-    ignite/cache/query/query_text.h \
-    ignite/ignite.h \
-    ignite/ignite_binding.h \
-    ignite/ignite_binding_context.h \
-    ignite/ignite_configuration.h \
-    ignite/ignition.h \
-    ignite/impl/binary/binary_type_updater_impl.h \
-    ignite/impl/bindings.h \
-    ignite/impl/cache/cache_entry_processor_holder.h \
-    ignite/impl/cache/cache_impl.h \
-    ignite/impl/cache/event/cache_entry_event_filter_base.h \
-    ignite/impl/cache/event/cache_entry_event_filter_holder.h \
-    ignite/impl/cache/query/continuous/continuous_query_handle_impl.h \
-    ignite/impl/cache/query/continuous/continuous_query_impl.h \
-    ignite/impl/cache/query/query_argument.h \
-    ignite/impl/cache/query/query_batch.h \
-    ignite/impl/cache/query/query_fields_row_impl.h \
-    ignite/impl/cache/query/query_impl.h \
-    ignite/impl/cluster/cluster_group_impl.h \
-    ignite/impl/handle_registry.h \
-    ignite/impl/ignite_binding_impl.h \
-    ignite/impl/ignite_environment.h \
-    ignite/impl/ignite_impl.h \
-    ignite/impl/interop/interop_external_memory.h \
-    ignite/impl/interop/interop_target.h \
-    ignite/impl/module_manager.h \
-    ignite/impl/operations.h \
-    ignite/impl/transactions/transactions_impl.h \
-    ignite/impl/transactions/transaction_impl.h \
-    ignite/transactions/transaction.h \
-    ignite/transactions/transactions.h \
-    ignite/transactions/transaction_consts.h \
-    ignite/transactions/transaction_metrics.h
+	ignite/cache/cache.h \
+	ignite/cache/cache_entry.h \
+	ignite/cache/cache_entry_processor.h \
+	ignite/cache/cache_peek_mode.h \
+	ignite/cache/event/cache_entry_event.h \
+	ignite/cache/event/cache_entry_event_filter.h \
+	ignite/cache/event/cache_entry_event_listener.h \
+	ignite/cache/mutable_cache_entry.h \
+	ignite/cache/query/continuous/continuous_query.h \
+	ignite/cache/query/continuous/continuous_query_handle.h \
+	ignite/cache/query/query.h \
+	ignite/cache/query/query_cursor.h \
+	ignite/cache/query/query_fields_cursor.h \
+	ignite/cache/query/query_fields_row.h \
+	ignite/cache/query/query_scan.h \
+	ignite/cache/query/query_sql.h \
+	ignite/cache/query/query_sql_fields.h \
+	ignite/cache/query/query_text.h \
+	ignite/compute/compute.h \
+	ignite/compute/compute_func.h \
+	ignite/ignite.h \
+	ignite/ignite_binding.h \
+	ignite/ignite_binding_context.h \
+	ignite/ignite_configuration.h \
+	ignite/ignition.h \
+	ignite/impl/binary/binary_type_updater_impl.h \
+	ignite/impl/bindings.h \
+	ignite/impl/cache/cache_entry_processor_holder.h \
+	ignite/impl/cache/cache_impl.h \
+	ignite/impl/cache/event/cache_entry_event_filter_base.h \
+	ignite/impl/cache/event/cache_entry_event_filter_holder.h \
+	ignite/impl/cache/query/continuous/continuous_query_handle_impl.h \
+	ignite/impl/cache/query/continuous/continuous_query_impl.h \
+	ignite/impl/cache/query/query_argument.h \
+	ignite/impl/cache/query/query_batch.h \
+	ignite/impl/cache/query/query_fields_row_impl.h \
+	ignite/impl/cache/query/query_impl.h \
+	ignite/impl/cluster/cluster_group_impl.h \
+	ignite/impl/compute/cancelable_impl.h \
+	ignite/impl/compute/compute_impl.h \
+	ignite/impl/compute/compute_job_holder.h \
+	ignite/impl/compute/compute_job_result.h \
+	ignite/impl/compute/compute_task_holder.h \
+	ignite/impl/handle_registry.h \
+	ignite/impl/ignite_binding_impl.h \
+	ignite/impl/ignite_environment.h \
+	ignite/impl/ignite_impl.h \
+	ignite/impl/interop/interop_external_memory.h \
+	ignite/impl/interop/interop_target.h \
+	ignite/impl/module_manager.h \
+	ignite/impl/operations.h \
+	ignite/impl/transactions/transactions_impl.h \
+	ignite/impl/transactions/transaction_impl.h \
+	ignite/transactions/transaction.h \
+	ignite/transactions/transactions.h \
+	ignite/transactions/transaction_consts.h \
+	ignite/transactions/transaction_metrics.h
 
 uninstall-hook:
 	if [ -d ${includedir}/ignite ]; then find ${includedir}/ignite -type d -empty -delete; fi

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9c96de5/modules/platforms/cpp/core/include/ignite/cache/cache.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/cache/cache.h b/modules/platforms/cpp/core/include/ignite/cache/cache.h
index e37e806..b16d7f5 100644
--- a/modules/platforms/cpp/core/include/ignite/cache/cache.h
+++ b/modules/platforms/cpp/core/include/ignite/cache/cache.h
@@ -1529,7 +1529,7 @@ namespace ignite
              * @code{.cpp}
              * IGNITE_EXPORTED_CALL void IgniteModuleInit(ignite::IgniteBindingContext& context)
              * {
-             *     IgniteBinding binding = context.GetBingding();
+             *     IgniteBinding binding = context.GetBinding();
              *
              *     binding.RegisterCacheEntryProcessor<MyProcessor1>();
              *     binding.RegisterCacheEntryProcessor<MyProcessor2>();
@@ -1582,7 +1582,7 @@ namespace ignite
              * @code{.cpp}
              * IGNITE_EXPORTED_CALL void IgniteModuleInit(ignite::IgniteBindingContext& context)
              * {
-             *     IgniteBinding binding = context.GetBingding();
+             *     IgniteBinding binding = context.GetBinding();
              *
              *     binding.RegisterCacheEntryProcessor<MyProcessor1>();
              *     binding.RegisterCacheEntryProcessor<MyProcessor2>();

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9c96de5/modules/platforms/cpp/core/include/ignite/cache/query/query.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/cache/query/query.h b/modules/platforms/cpp/core/include/ignite/cache/query/query.h
index b2991e4..a5a5c1f 100644
--- a/modules/platforms/cpp/core/include/ignite/cache/query/query.h
+++ b/modules/platforms/cpp/core/include/ignite/cache/query/query.h
@@ -23,7 +23,6 @@
 #ifndef _IGNITE_CACHE_QUERY_QUERY
 #define _IGNITE_CACHE_QUERY_QUERY
 
-#include "ignite/cache/query/query_argument.h"
 #include "ignite/cache/query/query_cursor.h"
 #include "ignite/cache/query/query_scan.h"
 #include "ignite/cache/query/query_sql.h"

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9c96de5/modules/platforms/cpp/core/include/ignite/compute/compute.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/compute/compute.h b/modules/platforms/cpp/core/include/ignite/compute/compute.h
new file mode 100644
index 0000000..b079569
--- /dev/null
+++ b/modules/platforms/cpp/core/include/ignite/compute/compute.h
@@ -0,0 +1,136 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * @file
+ * Declares ignite::compute::Compute class.
+ */
+
+#ifndef _IGNITE_COMPUTE_COMPUTE
+#define _IGNITE_COMPUTE_COMPUTE
+
+#include <ignite/common/common.h>
+
+#include <ignite/ignite_error.h>
+#include <ignite/future.h>
+#include <ignite/compute/compute_func.h>
+
+#include <ignite/impl/compute/compute_impl.h>
+
+namespace ignite
+{
+    namespace compute
+    {
+        /**
+         * Defines compute grid functionality for executing tasks and closures
+         * over nodes in the ClusterGroup. Instance of Compute is obtained from
+         * Ignite as follows:
+         * @code{.cpp}
+         * Ignite node = Ignition::Get();
+         *
+         * // Compute over all nodes in the cluster.
+         * Compute c = node.GetCompute();
+         * @endcode
+         *
+         * @par Load Balancing
+         * In all cases other than <tt>Broadcast(...)</tt>, Ignite must select
+         * a node for a computation to be executed. The node will be selected
+         * based on the underlying \c LoadBalancingSpi, which by default
+         * sequentially picks next available node from the underlying cluster
+         * group. Other load balancing policies, such as \c random or
+         * \c adaptive, can be configured as well by selecting a different
+         * load balancing SPI in Ignite configuration.
+         *
+         * @par Fault Tolerance
+         * Ignite guarantees that as long as there is at least one grid node
+         * standing, every job will be executed. Jobs will automatically
+         * failover to another node if a remote node crashed or has rejected
+         * execution due to lack of resources. By default, in case of failover,
+         * next load balanced node will be picked for job execution. Also jobs
+         * will never be re-routed to the nodes they have failed on. This
+         * behavior can be changed by configuring any of the existing or a
+         * custom FailoverSpi in grid configuration.
+         *
+         * @par Computation SPIs
+         * Note that regardless of which method is used for executing
+         * computations, all relevant SPI implementations configured for this
+         * compute instance will be used (i.e. failover, load balancing,
+         * collision resolution, checkpoints, etc.).
+         */
+        class IGNITE_IMPORT_EXPORT Compute
+        {
+        public:
+            /**
+             * Constructor.
+             *
+             * Internal method. Should not be used by user.
+             *
+             * @param impl Implementation.
+             */
+            Compute(common::concurrent::SharedPointer<impl::compute::ComputeImpl> impl) :
+                impl(impl)
+            {
+                // No-op.
+            }
+
+            /**
+             * Calls provided ComputeFunc on a node within the underlying
+             * cluster group.
+             *
+             * @tparam R Call return type. BinaryType should be specialized for
+             *  the type if it is not primitive. Should not be void. For
+             *  non-returning methods see Compute::Run().
+             * @tparam F Compute function type. Should implement ComputeFunc
+             *  class.
+             * @param func Compute function to call.
+             * @return Computation result.
+             * @throw IgniteError in case of error.
+             */
+            template<typename R, typename F>
+            R Call(const F& func)
+            {
+                return impl.Get()->CallAsync<R, F>(func).GetValue();
+            }
+
+            /**
+             * Asyncronuously calls provided ComputeFunc on a node within
+             * the underlying cluster group.
+             *
+             * @tparam R Call return type. BinaryType should be specialized for
+             *  the type if it is not primitive. Should not be void. For
+             *  non-returning methods see Compute::Run().
+             * @tparam F Compute function type. Should implement ComputeFunc
+             *  class.
+             * @param func Compute function to call.
+             * @return Future that can be used to access computation result once
+             *  it's ready.
+             * @throw IgniteError in case of error.
+             */
+            template<typename R, typename F>
+            Future<R> CallAsync(const F& func)
+            {
+                return impl.Get()->CallAsync<R, F>(func);
+            }
+
+        private:
+            /** Implementation. */
+            common::concurrent::SharedPointer<impl::compute::ComputeImpl> impl;
+        };
+    }
+}
+
+#endif //_IGNITE_COMPUTE_COMPUTE

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9c96de5/modules/platforms/cpp/core/include/ignite/compute/compute_func.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/compute/compute_func.h b/modules/platforms/cpp/core/include/ignite/compute/compute_func.h
new file mode 100644
index 0000000..93a7b4f
--- /dev/null
+++ b/modules/platforms/cpp/core/include/ignite/compute/compute_func.h
@@ -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.
+ */
+
+/**
+ * @file
+ * Declares ignite::compute::ComputeFunc class template.
+ */
+
+#ifndef _IGNITE_COMPUTE_COMPUTE_FUNC
+#define _IGNITE_COMPUTE_COMPUTE_FUNC
+
+namespace ignite
+{
+    class IgniteBinding;
+
+    namespace compute
+    {
+        /**
+         * Interface for a simple compute function that can be serialized and
+         * called on the remote nodes. ignite::binary::BinaryType class template
+         * should be specialized for any class, inheriting from this class.
+         *
+         * @tparam R Call return type. BinaryType should be specialized for the
+         *  type if it is not primitive.
+         */
+        template<typename R>
+        class ComputeFunc
+        {
+            friend class ignite::IgniteBinding;
+
+            typedef R ReturnType;
+        public:
+            /**
+             * Destructor.
+             */
+            virtual ~ComputeFunc()
+            {
+                // No-op.
+            }
+
+            /**
+             * Called upon execution by compute.
+             *
+             * @return Computation result.
+             */
+            virtual R Call() = 0;
+        };
+    }
+}
+
+#endif //_IGNITE_COMPUTE_COMPUTE_FUNC

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9c96de5/modules/platforms/cpp/core/include/ignite/ignite.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/ignite.h b/modules/platforms/cpp/core/include/ignite/ignite.h
index d96f81f..b3b06f0 100644
--- a/modules/platforms/cpp/core/include/ignite/ignite.h
+++ b/modules/platforms/cpp/core/include/ignite/ignite.h
@@ -23,10 +23,12 @@
 #ifndef _IGNITE_IGNITE
 #define _IGNITE_IGNITE
 
-#include "ignite/cache/cache.h"
-#include "ignite/transactions/transactions.h"
-#include "ignite/impl/ignite_impl.h"
-#include "ignite/ignite_configuration.h"
+#include <ignite/impl/ignite_impl.h>
+
+#include <ignite/ignite_configuration.h>
+#include <ignite/cache/cache.h>
+#include <ignite/transactions/transactions.h>
+#include <ignite/compute/compute.h>
 
 namespace ignite
 {
@@ -189,6 +191,15 @@ namespace ignite
         transactions::Transactions GetTransactions();
 
         /**
+         * Get compute.
+         *
+         * This method should only be called on the valid instance.
+         *
+         * @return Compute class instance.
+         */
+        compute::Compute GetCompute();
+
+        /**
          * Get ignite binding.
          *
          * This method should only be used on the valid instance.

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9c96de5/modules/platforms/cpp/core/include/ignite/ignite_binding.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/ignite_binding.h b/modules/platforms/cpp/core/include/ignite/ignite_binding.h
index 344bdf6..fe060c1 100644
--- a/modules/platforms/cpp/core/include/ignite/ignite_binding.h
+++ b/modules/platforms/cpp/core/include/ignite/ignite_binding.h
@@ -121,6 +121,31 @@ namespace ignite
         }
 
         /**
+         * Register type as Compute function.
+         *
+         * Registred type should be a child of ignite::compute::ComputeFunc
+         * class.
+         */
+        template<typename F>
+        void RegisterComputeFunc()
+        {
+            impl::IgniteBindingImpl *im = impl.Get();
+
+            int32_t typeId = binary::BinaryType<F>::GetTypeId();
+
+            if (im)
+            {
+                im->RegisterCallback(impl::IgniteBindingImpl::CallbackType::COMPUTE_JOB_CREATE,
+                    typeId, impl::binding::ComputeJobCreate<F, typename F::ReturnType>);
+            }
+            else
+            {
+                throw IgniteError(IgniteError::IGNITE_ERR_GENERIC,
+                    "Instance is not usable (did you check for error?).");
+            }
+        }
+
+        /**
          * Check if the instance is valid.
          *
          * Invalid instance can be returned if some of the previous operations

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9c96de5/modules/platforms/cpp/core/include/ignite/ignite_binding_context.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/ignite_binding_context.h b/modules/platforms/cpp/core/include/ignite/ignite_binding_context.h
index 4d8a7a7..244b255 100644
--- a/modules/platforms/cpp/core/include/ignite/ignite_binding_context.h
+++ b/modules/platforms/cpp/core/include/ignite/ignite_binding_context.h
@@ -48,7 +48,7 @@ namespace ignite
          *
          * @return IgniteBinding instance.
          */
-        IgniteBinding GetBingding() const
+        IgniteBinding GetBinding() const
         {
             return binding;
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9c96de5/modules/platforms/cpp/core/include/ignite/impl/bindings.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/impl/bindings.h b/modules/platforms/cpp/core/include/ignite/impl/bindings.h
index c1ada12..357b0f0 100644
--- a/modules/platforms/cpp/core/include/ignite/impl/bindings.h
+++ b/modules/platforms/cpp/core/include/ignite/impl/bindings.h
@@ -24,6 +24,7 @@
 #include <ignite/impl/ignite_environment.h>
 #include <ignite/impl/cache/query/continuous/continuous_query_impl.h>
 #include <ignite/impl/cache/cache_entry_processor_holder.h>
+#include <ignite/impl/compute/compute_task_holder.h>
 
 namespace ignite
 {
@@ -88,6 +89,29 @@ namespace ignite
 
                 return 0;
             }
+
+            /**
+             * Binding for compute job creation.
+             *
+             * @tparam F The job type.
+             * @tparam R The job return type.
+             *
+             * @param reader Reader.
+             * @param env Environment.
+             * @return Handle for the job.
+             */
+            template<typename F, typename R>
+            int64_t ComputeJobCreate(binary::BinaryReaderImpl& reader, binary::BinaryWriterImpl&, IgniteEnvironment& env)
+            {
+                using namespace common::concurrent;
+                using namespace compute;
+
+                F job = reader.ReadObject<F>();
+
+                SharedPointer<ComputeJobHolder> jobPtr(new ComputeJobHolderImpl<F, R>(job));
+
+                return env.GetHandleRegistry().Allocate(jobPtr);
+            }
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9c96de5/modules/platforms/cpp/core/include/ignite/impl/cluster/cluster_group_impl.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/impl/cluster/cluster_group_impl.h b/modules/platforms/cpp/core/include/ignite/impl/cluster/cluster_group_impl.h
index 2bff0d8..3cfd700 100644
--- a/modules/platforms/cpp/core/include/ignite/impl/cluster/cluster_group_impl.h
+++ b/modules/platforms/cpp/core/include/ignite/impl/cluster/cluster_group_impl.h
@@ -22,6 +22,7 @@
 #include <ignite/jni/java.h>
 
 #include <ignite/impl/interop/interop_target.h>
+#include <ignite/impl/compute/compute_impl.h>
 
 namespace ignite
 {
@@ -29,13 +30,19 @@ namespace ignite
     {
         namespace cluster
         {
+            /* Forward declaration. */
+            class ClusterGroupImpl;
+
+            /* Shared pointer. */
+            typedef common::concurrent::SharedPointer<ClusterGroupImpl> SP_ClusterGroupImpl;
+
             /**
              * Cluster group implementation.
              */
             class IGNITE_FRIEND_EXPORT ClusterGroupImpl : private interop::InteropTarget
             {
                 typedef common::concurrent::SharedPointer<IgniteEnvironment> SP_IgniteEnvironment;
-                typedef common::concurrent::SharedPointer<ClusterGroupImpl> SP_ClusterGroupImpl;
+                typedef common::concurrent::SharedPointer<compute::ComputeImpl> SP_ComputeImpl;
             public:
                 /**
                  * Constructor used to create new instance.
@@ -53,12 +60,20 @@ namespace ignite
                 /**
                  * Get server nodes cluster group implementation.
                  *
-                 * @param err Error.
                  * @return Server nodes cluster group implementation.
                  */
-                SP_ClusterGroupImpl ForServers(IgniteError& err);
+                SP_ClusterGroupImpl ForServers();
+
+                /**
+                 * Get compute instance over this cluster group.
+                 *
+                 * @return Compute instance.
+                 */
+                SP_ComputeImpl GetCompute();
 
             private:
+                IGNITE_NO_COPY_ASSIGNMENT(ClusterGroupImpl);
+
                 /**
                  * Make cluster group implementation using java reference and
                  * internal state of this cluster group.
@@ -68,7 +83,15 @@ namespace ignite
                  */
                 SP_ClusterGroupImpl FromTarget(jobject javaRef);
 
-                IGNITE_NO_COPY_ASSIGNMENT(ClusterGroupImpl)
+                /**
+                 * Gets instance of compute internally.
+                 *
+                 * @return Instance of compute.
+                 */
+                SP_ComputeImpl InternalGetCompute();
+
+                /** Compute for the cluster group. */
+                SP_ComputeImpl computeImpl;
             };
         }
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9c96de5/modules/platforms/cpp/core/include/ignite/impl/compute/cancelable_impl.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/impl/compute/cancelable_impl.h b/modules/platforms/cpp/core/include/ignite/impl/compute/cancelable_impl.h
new file mode 100644
index 0000000..828ecba
--- /dev/null
+++ b/modules/platforms/cpp/core/include/ignite/impl/compute/cancelable_impl.h
@@ -0,0 +1,70 @@
+/*
+ * 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.
+ */
+
+/**
+ * @file
+ * Declares ignite::impl::compute::CancelableImpl class.
+ */
+
+#ifndef _IGNITE_IMPL_COMPUTE_CANCELABLE_IMPL
+#define _IGNITE_IMPL_COMPUTE_CANCELABLE_IMPL
+
+#include <ignite/common/common.h>
+#include <ignite/common/cancelable.h>
+#include <ignite/impl/interop/interop_target.h>
+
+namespace ignite
+{
+    namespace impl
+    {
+        namespace compute
+        {
+            /**
+             * Compute implementation.
+             */
+            class IGNITE_IMPORT_EXPORT CancelableImpl : public interop::InteropTarget, public common::Cancelable
+            {
+            public:
+                /**
+                 * Constructor.
+                 *
+                 * @param env Environment.
+                 * @param javaRef Java object reference.
+                 */
+                CancelableImpl(common::concurrent::SharedPointer<IgniteEnvironment> env, jobject javaRef);
+
+                /**
+                 * Destructor.
+                 */
+                virtual ~CancelableImpl()
+                {
+                    // No-op.
+                }
+
+                /**
+                 * Cancels the operation.
+                 */
+                virtual void Cancel();
+
+            private:
+                IGNITE_NO_COPY_ASSIGNMENT(CancelableImpl);
+            };
+        }
+    }
+}
+
+#endif //_IGNITE_IMPL_COMPUTE_CANCELABLE_IMPL

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9c96de5/modules/platforms/cpp/core/include/ignite/impl/compute/compute_impl.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/impl/compute/compute_impl.h b/modules/platforms/cpp/core/include/ignite/impl/compute/compute_impl.h
new file mode 100644
index 0000000..389c571
--- /dev/null
+++ b/modules/platforms/cpp/core/include/ignite/impl/compute/compute_impl.h
@@ -0,0 +1,118 @@
+/*
+ * 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.
+ */
+
+/**
+ * @file
+ * Declares ignite::impl::compute::ComputeImpl class.
+ */
+
+#ifndef _IGNITE_IMPL_COMPUTE_COMPUTE_IMPL
+#define _IGNITE_IMPL_COMPUTE_COMPUTE_IMPL
+
+#include <ignite/common/common.h>
+#include <ignite/common/promise.h>
+#include <ignite/impl/interop/interop_target.h>
+#include <ignite/impl/compute/compute_task_holder.h>
+#include <ignite/impl/compute/cancelable_impl.h>
+
+#include <ignite/ignite_error.h>
+
+namespace ignite
+{
+    namespace impl
+    {
+        namespace compute
+        {
+            /**
+             * Compute implementation.
+             */
+            class IGNITE_IMPORT_EXPORT ComputeImpl : public interop::InteropTarget
+            {
+            public:
+                /**
+                 * Operation type.
+                 */
+                struct Operation
+                {
+                    enum Type
+                    {
+                        Unicast = 5
+                    };
+                };
+
+                /**
+                 * Constructor.
+                 *
+                 * @param env Environment.
+                 * @param javaRef Java object reference.
+                 */
+                ComputeImpl(common::concurrent::SharedPointer<IgniteEnvironment> env, jobject javaRef);
+
+                /**
+                 * Asyncronuously calls provided ComputeFunc on a node within
+                 * the underlying cluster group.
+                 *
+                 * @tparam F Compute function type. Should implement ComputeFunc
+                 *  class.
+                 * @tparam R Call return type. BinaryType should be specialized for
+                 *  the type if it is not primitive. Should not be void. For
+                 *  non-returning methods see Compute::Run().
+                 * @param func Compute function to call.
+                 * @return Future that can be used to acess computation result once
+                 *  it's ready.
+                 * @throw IgniteError in case of error.
+                 */
+                template<typename R, typename F>
+                Future<R> CallAsync(const F& func)
+                {
+                    common::concurrent::SharedPointer<interop::InteropMemory> mem = GetEnvironment().AllocateMemory();
+                    interop::InteropOutputStream out(mem.Get());
+                    binary::BinaryWriterImpl writer(&out, GetEnvironment().GetTypeManager());
+
+                    common::concurrent::SharedPointer<ComputeJobHolder> job(new ComputeJobHolderImpl<F, R>(func));
+
+                    int64_t jobHandle = GetEnvironment().GetHandleRegistry().Allocate(job);
+
+                    ComputeTaskHolderImpl<F, R>* taskPtr = new ComputeTaskHolderImpl<F, R>(jobHandle);
+                    common::concurrent::SharedPointer<ComputeTaskHolder> task(taskPtr);
+
+                    int64_t taskHandle = GetEnvironment().GetHandleRegistry().Allocate(task);
+
+                    writer.WriteInt64(taskHandle);
+                    writer.WriteInt32(1);
+                    writer.WriteInt64(jobHandle);
+                    writer.WriteObject<F>(func);
+
+                    out.Synchronize();
+
+                    jobject target = InStreamOutObject(Operation::Unicast, *mem.Get());
+                    std::auto_ptr<common::Cancelable> cancelable(new CancelableImpl(GetEnvironmentPointer(), target));
+
+                    common::Promise<R>& promise = taskPtr->GetPromise();
+                    promise.SetCancelTarget(cancelable);
+
+                    return promise.GetFuture();
+                }
+
+            private:
+                IGNITE_NO_COPY_ASSIGNMENT(ComputeImpl);
+            };
+        }
+    }
+}
+
+#endif //_IGNITE_IMPL_COMPUTE_COMPUTE_IMPL


[16/19] ignite git commit: IGNITE-5337: Renamed C++ example executables

Posted by sb...@apache.org.
IGNITE-5337: Renamed C++ example executables


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

Branch: refs/heads/ignite-5075
Commit: a8586118ff4acbbd2ff9489edf954a187e0892d7
Parents: 73ae278
Author: Igor Sapego <is...@gridgain.com>
Authored: Tue May 30 20:13:02 2017 +0300
Committer: Igor Sapego <is...@gridgain.com>
Committed: Tue May 30 20:13:02 2017 +0300

----------------------------------------------------------------------
 modules/platforms/cpp/examples/Makefile.am      |   2 +-
 modules/platforms/cpp/examples/configure.ac     |   2 +-
 .../cpp/examples/odbc-example/Makefile.am       |  12 +-
 .../cpp/examples/project/vs/ignite-examples.sln |  12 +-
 .../cpp/examples/put-get-example/Makefile.am    |  58 +++++++++
 .../put-get-example/config/example-cache.xml    |  67 ++++++++++
 .../project/vs/put-get-example.vcxproj          | 110 ++++++++++++++++
 .../project/vs/put-get-example.vcxproj.filters  |  38 ++++++
 .../put-get-example/src/put_get_example.cpp     | 126 +++++++++++++++++++
 .../cpp/examples/putget-example/Makefile.am     |  58 ---------
 .../putget-example/config/example-cache.xml     |  67 ----------
 .../project/vs/putget-example.vcxproj           | 110 ----------------
 .../project/vs/putget-example.vcxproj.filters   |  38 ------
 .../putget-example/src/putget_example.cpp       | 126 -------------------
 .../cpp/examples/query-example/Makefile.am      |  12 +-
 15 files changed, 419 insertions(+), 419 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a8586118/modules/platforms/cpp/examples/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/examples/Makefile.am b/modules/platforms/cpp/examples/Makefile.am
index e1c7905..8cfb999 100644
--- a/modules/platforms/cpp/examples/Makefile.am
+++ b/modules/platforms/cpp/examples/Makefile.am
@@ -18,7 +18,7 @@
 ACLOCAL_AMFLAGS =-I m4
 
 SUBDIRS = \
-    putget-example \
+    put-get-example \
     odbc-example \
     query-example \
 	continuous-query-example \

http://git-wip-us.apache.org/repos/asf/ignite/blob/a8586118/modules/platforms/cpp/examples/configure.ac
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/examples/configure.ac b/modules/platforms/cpp/examples/configure.ac
index 3350d40..7aef863 100644
--- a/modules/platforms/cpp/examples/configure.ac
+++ b/modules/platforms/cpp/examples/configure.ac
@@ -53,7 +53,7 @@ fi
 AC_CONFIG_FILES([ \
     Makefile \
     include/Makefile \
-    putget-example/Makefile \
+    put-get-example/Makefile \
     odbc-example/Makefile \
     query-example/Makefile \
     continuous-query-example/Makefile \

http://git-wip-us.apache.org/repos/asf/ignite/blob/a8586118/modules/platforms/cpp/examples/odbc-example/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/examples/odbc-example/Makefile.am b/modules/platforms/cpp/examples/odbc-example/Makefile.am
index f3c9a50..88645d3 100644
--- a/modules/platforms/cpp/examples/odbc-example/Makefile.am
+++ b/modules/platforms/cpp/examples/odbc-example/Makefile.am
@@ -17,7 +17,7 @@
 
 ACLOCAL_AMFLAGS =-I m4
 
-noinst_PROGRAMS = ignite-odbcexample
+noinst_PROGRAMS = ignite-odbc-example
 
 AM_CPPFLAGS = \
     -I@top_srcdir@/include \
@@ -38,22 +38,22 @@ AM_CXXFLAGS = \
     -Wall \
     -std=c++03
 
-ignite_odbcexample_LDADD = \
+ignite_odbc_example_LDADD = \
     @top_srcdir@/../core/libignite.la \
     -lodbc \
     -lpthread
 
-ignite_odbcexample_LDFLAGS = \
+ignite_odbc_example_LDFLAGS = \
     -static-libtool-libs
 
-ignite_odbcexample_SOURCES = \
+ignite_odbc_example_SOURCES = \
     src/odbc_example.cpp
 
 run-check: check
-	./ignite-odbcexample -p
+	./ignite-odbc-example -p
 
 clean-local: clean-check
 	$(RM) *.gcno *.gcda
 
 clean-check:
-	$(RM) $(ignite_odbcexample_OBJECTS)
+	$(RM) $(ignite_odbc_example_OBJECTS)

http://git-wip-us.apache.org/repos/asf/ignite/blob/a8586118/modules/platforms/cpp/examples/project/vs/ignite-examples.sln
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/examples/project/vs/ignite-examples.sln b/modules/platforms/cpp/examples/project/vs/ignite-examples.sln
index d839f09..a1865e3 100644
--- a/modules/platforms/cpp/examples/project/vs/ignite-examples.sln
+++ b/modules/platforms/cpp/examples/project/vs/ignite-examples.sln
@@ -1,24 +1,20 @@
 
 Microsoft Visual Studio Solution File, Format Version 11.00
 # Visual Studio 2010
-Project("{8BC9CEB8-8B4A-11D0-8D11-00A0C91BC942}") = "putget-example", "..\..\putget-example\project\vs\putget-example.vcxproj", "{34935DEC-80FC-4168-AA52-3DBFF4F79B6B}"
-EndProject
 Project("{8BC9CEB8-8B4A-11D0-8D11-00A0C91BC942}") = "odbc-example", "..\..\odbc-example\project\vs\odbc-example.vcxproj", "{56839DFF-6C03-416B-BC5F-DDC6EBF8512D}"
 EndProject
 Project("{8BC9CEB8-8B4A-11D0-8D11-00A0C91BC942}") = "query-example", "..\..\query-example\project\vs\query-example.vcxproj", "{9FB34AB4-01DD-4C6F-99BF-681019D0E4DD}"
 EndProject
 Project("{8BC9CEB8-8B4A-11D0-8D11-00A0C91BC942}") = "continuous-query-example", "..\..\continuous-query-example\project\vs\continuous-query-example.vcxproj", "{73BB124A-0CD4-4961-A6CD-61F9C71028A6}"
 EndProject
+Project("{8BC9CEB8-8B4A-11D0-8D11-00A0C91BC942}") = "put-get-example", "..\..\put-get-example\project\vs\put-get-example.vcxproj", "{34935DEC-80FC-4168-AA52-3DBFF4F79B6B}"
+EndProject
 Global
 	GlobalSection(SolutionConfigurationPlatforms) = preSolution
 		Release|x64 = Release|x64
 		Release|x86 = Release|x86
 	EndGlobalSection
 	GlobalSection(ProjectConfigurationPlatforms) = postSolution
-		{34935DEC-80FC-4168-AA52-3DBFF4F79B6B}.Release|x64.ActiveCfg = Release|x64
-		{34935DEC-80FC-4168-AA52-3DBFF4F79B6B}.Release|x64.Build.0 = Release|x64
-		{34935DEC-80FC-4168-AA52-3DBFF4F79B6B}.Release|x86.ActiveCfg = Release|Win32
-		{34935DEC-80FC-4168-AA52-3DBFF4F79B6B}.Release|x86.Build.0 = Release|Win32
 		{56839DFF-6C03-416B-BC5F-DDC6EBF8512D}.Release|x64.ActiveCfg = Release|x64
 		{56839DFF-6C03-416B-BC5F-DDC6EBF8512D}.Release|x64.Build.0 = Release|x64
 		{56839DFF-6C03-416B-BC5F-DDC6EBF8512D}.Release|x86.ActiveCfg = Release|Win32
@@ -31,6 +27,10 @@ Global
 		{73BB124A-0CD4-4961-A6CD-61F9C71028A6}.Release|x64.Build.0 = Release|x64
 		{73BB124A-0CD4-4961-A6CD-61F9C71028A6}.Release|x86.ActiveCfg = Release|Win32
 		{73BB124A-0CD4-4961-A6CD-61F9C71028A6}.Release|x86.Build.0 = Release|Win32
+		{34935DEC-80FC-4168-AA52-3DBFF4F79B6B}.Release|x64.ActiveCfg = Release|x64
+		{34935DEC-80FC-4168-AA52-3DBFF4F79B6B}.Release|x64.Build.0 = Release|x64
+		{34935DEC-80FC-4168-AA52-3DBFF4F79B6B}.Release|x86.ActiveCfg = Release|Win32
+		{34935DEC-80FC-4168-AA52-3DBFF4F79B6B}.Release|x86.Build.0 = Release|Win32
 	EndGlobalSection
 	GlobalSection(SolutionProperties) = preSolution
 		HideSolutionNode = FALSE

http://git-wip-us.apache.org/repos/asf/ignite/blob/a8586118/modules/platforms/cpp/examples/put-get-example/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/examples/put-get-example/Makefile.am b/modules/platforms/cpp/examples/put-get-example/Makefile.am
new file mode 100644
index 0000000..27154c5
--- /dev/null
+++ b/modules/platforms/cpp/examples/put-get-example/Makefile.am
@@ -0,0 +1,58 @@
+##
+## 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.
+##
+
+ACLOCAL_AMFLAGS =-I m4
+
+noinst_PROGRAMS = ignite-put-get-example
+
+AM_CPPFLAGS = \
+    -I@top_srcdir@/include \
+    -I@top_srcdir@/../core/include \
+    -I@top_srcdir@/../core/os/linux/include \
+    -I@top_srcdir@/../common/include \
+    -I@top_srcdir@/../common/os/linux/include \
+    -I@top_srcdir@/../binary/include \
+    -I@top_srcdir@/../jni/include \
+    -I@top_srcdir@/../jni/os/linux/include \
+    -I$(JAVA_HOME)/include \
+    -I$(JAVA_HOME)/include/linux \
+    -DIGNITE_IMPL \
+    -D__STDC_LIMIT_MACROS \
+    -D__STDC_CONSTANT_MACROS
+
+AM_CXXFLAGS = \
+    -Wall \
+    -std=c++03
+
+ignite_put_get_example_LDADD = \
+    @top_srcdir@/../core/libignite.la \
+    -lpthread
+
+ignite_put_get_example_LDFLAGS = \
+    -static-libtool-libs
+
+ignite_put_get_example_SOURCES = \
+    src/put_get_example.cpp
+
+run-check: check
+	./ignite-put-get-example -p
+
+clean-local: clean-check
+	$(RM) *.gcno *.gcda
+
+clean-check:
+	$(RM) $(ignite_put_get_example_OBJECTS)

http://git-wip-us.apache.org/repos/asf/ignite/blob/a8586118/modules/platforms/cpp/examples/put-get-example/config/example-cache.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/examples/put-get-example/config/example-cache.xml b/modules/platforms/cpp/examples/put-get-example/config/example-cache.xml
new file mode 100644
index 0000000..af523cd
--- /dev/null
+++ b/modules/platforms/cpp/examples/put-get-example/config/example-cache.xml
@@ -0,0 +1,67 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xmlns:util="http://www.springframework.org/schema/util"
+       xsi:schemaLocation="
+        http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd
+        http://www.springframework.org/schema/util
+        http://www.springframework.org/schema/util/spring-util.xsd">
+    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+        <!-- Set to true to enable distributed class loading for examples, default is false. -->
+        <property name="peerClassLoadingEnabled" value="true"/>
+
+        <property name="cacheConfiguration">
+            <list>
+                <!--
+                    Partitioned cache example configuration with binary objects enabled.
+                -->
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="atomic"/>
+                    <property name="atomicityMode" value="ATOMIC"/>
+                    <property name="backups" value="1"/>
+                </bean>
+            </list>
+        </property>
+
+        <!-- Explicitly configure TCP discovery SPI to provide list of initial nodes. -->
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="ipFinder">
+                    <!--
+                        Ignite provides several options for automatic discovery that can be used
+                        instead os static IP based discovery.
+                    -->
+                    <!-- Uncomment static IP finder to enable static-based discovery of initial nodes. -->
+                    <!--<bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">-->
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.TcpDiscoveryMulticastIpFinder">
+                        <property name="addresses">
+                            <list>
+                                <!-- In distributed environment, replace with actual host IP address. -->
+                                <value>127.0.0.1:47500..47501</value>
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+            </bean>
+        </property>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/a8586118/modules/platforms/cpp/examples/put-get-example/project/vs/put-get-example.vcxproj
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/examples/put-get-example/project/vs/put-get-example.vcxproj b/modules/platforms/cpp/examples/put-get-example/project/vs/put-get-example.vcxproj
new file mode 100644
index 0000000..cfb160f
--- /dev/null
+++ b/modules/platforms/cpp/examples/put-get-example/project/vs/put-get-example.vcxproj
@@ -0,0 +1,110 @@
+<?xml version="1.0" encoding="utf-8"?>
+<Project DefaultTargets="Build" ToolsVersion="4.0" xmlns="http://schemas.microsoft.com/developer/msbuild/2003">
+  <ItemGroup Label="ProjectConfigurations">
+    <ProjectConfiguration Include="Release|Win32">
+      <Configuration>Release</Configuration>
+      <Platform>Win32</Platform>
+    </ProjectConfiguration>
+    <ProjectConfiguration Include="Release|x64">
+      <Configuration>Release</Configuration>
+      <Platform>x64</Platform>
+    </ProjectConfiguration>
+  </ItemGroup>
+  <PropertyGroup Label="Globals">
+    <ProjectGuid>{34935DEC-80FC-4168-AA52-3DBFF4F79B6B}</ProjectGuid>
+    <Keyword>Win32Proj</Keyword>
+    <RootNamespace>igniteexamples</RootNamespace>
+  </PropertyGroup>
+  <Import Project="$(VCTargetsPath)\Microsoft.Cpp.Default.props" />
+  <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Release|x64'" Label="Configuration">
+    <ConfigurationType>Application</ConfigurationType>
+    <UseDebugLibraries>false</UseDebugLibraries>
+    <PlatformToolset>v100</PlatformToolset>
+    <WholeProgramOptimization>true</WholeProgramOptimization>
+    <CharacterSet>Unicode</CharacterSet>
+  </PropertyGroup>
+  <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Release|Win32'" Label="Configuration">
+    <ConfigurationType>Application</ConfigurationType>
+    <UseDebugLibraries>false</UseDebugLibraries>
+    <PlatformToolset>v100</PlatformToolset>
+    <WholeProgramOptimization>true</WholeProgramOptimization>
+    <CharacterSet>Unicode</CharacterSet>
+  </PropertyGroup>
+  <Import Project="$(VCTargetsPath)\Microsoft.Cpp.props" />
+  <ImportGroup Label="ExtensionSettings">
+  </ImportGroup>
+  <ImportGroup Condition="'$(Configuration)|$(Platform)'=='Release|x64'" Label="PropertySheets">
+    <Import Project="$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props" Condition="exists('$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props')" Label="LocalAppDataPlatform" />
+  </ImportGroup>
+  <ImportGroup Condition="'$(Configuration)|$(Platform)'=='Release|Win32'" Label="PropertySheets">
+    <Import Project="$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props" Condition="exists('$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props')" Label="LocalAppDataPlatform" />
+  </ImportGroup>
+  <PropertyGroup Label="UserMacros" />
+  <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Release|x64'">
+    <LinkIncremental>false</LinkIncremental>
+  </PropertyGroup>
+  <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Release|Win32'">
+    <LinkIncremental>false</LinkIncremental>
+  </PropertyGroup>
+  <ItemDefinitionGroup Condition="'$(Configuration)|$(Platform)'=='Release|x64'">
+    <ClCompile>
+      <WarningLevel>Level3</WarningLevel>
+      <PrecompiledHeader>
+      </PrecompiledHeader>
+      <Optimization>MaxSpeed</Optimization>
+      <FunctionLevelLinking>true</FunctionLevelLinking>
+      <IntrinsicFunctions>true</IntrinsicFunctions>
+      <PreprocessorDefinitions>WIN32;NDEBUG;_CONSOLE;_LIB;%(PreprocessorDefinitions)</PreprocessorDefinitions>
+      <AdditionalIncludeDirectories>$(JAVA_HOME)\include;$(JAVA_HOME)\include\win32;..\..\..\include;..\..\..\..\jni\os\win\include;..\..\..\..\jni\include;..\..\..\..\common\os\win\include;..\..\..\..\common\include;..\..\..\..\binary\include;..\..\..\..\core\os\win\include;..\..\..\..\core\include;%(AdditionalIncludeDirectories)</AdditionalIncludeDirectories>
+    </ClCompile>
+    <Link>
+      <SubSystem>Console</SubSystem>
+      <GenerateDebugInformation>true</GenerateDebugInformation>
+      <EnableCOMDATFolding>true</EnableCOMDATFolding>
+      <OptimizeReferences>true</OptimizeReferences>
+      <AdditionalDependencies>jvm.lib;ignite.common.lib;ignite.jni.lib;ignite.binary.lib;ignite.core.lib;%(AdditionalDependencies)</AdditionalDependencies>
+      <AdditionalLibraryDirectories>..\..\..\..\project\vs\$(Platform)\$(Configuration)\;$(JAVA_HOME)\lib;%(AdditionalLibraryDirectories)</AdditionalLibraryDirectories>
+    </Link>
+    <PostBuildEvent>
+      <Command>copy "$(ProjectDir)..\..\..\..\project\vs\$(Platform)\$(Configuration)\ignite.jni.dll" "$(OutDir)"
+copy "$(ProjectDir)..\..\..\..\project\vs\$(Platform)\$(Configuration)\ignite.core.dll" "$(OutDir)"</Command>
+    </PostBuildEvent>
+  </ItemDefinitionGroup>
+  <ItemDefinitionGroup Condition="'$(Configuration)|$(Platform)'=='Release|Win32'">
+    <ClCompile>
+      <WarningLevel>Level3</WarningLevel>
+      <PrecompiledHeader>
+      </PrecompiledHeader>
+      <Optimization>MaxSpeed</Optimization>
+      <FunctionLevelLinking>true</FunctionLevelLinking>
+      <IntrinsicFunctions>true</IntrinsicFunctions>
+      <PreprocessorDefinitions>WIN32;NDEBUG;_CONSOLE;_LIB;%(PreprocessorDefinitions)</PreprocessorDefinitions>
+      <AdditionalIncludeDirectories>$(JAVA_HOME)\include;$(JAVA_HOME)\include\win32;..\..\..\include;..\..\..\..\jni\os\win\include;..\..\..\..\jni\include;..\..\..\..\common\os\win\include;..\..\..\..\common\include;..\..\..\..\binary\include;..\..\..\..\core\os\win\include;..\..\..\..\core\include;%(AdditionalIncludeDirectories)</AdditionalIncludeDirectories>
+    </ClCompile>
+    <Link>
+      <SubSystem>Console</SubSystem>
+      <GenerateDebugInformation>true</GenerateDebugInformation>
+      <EnableCOMDATFolding>true</EnableCOMDATFolding>
+      <OptimizeReferences>true</OptimizeReferences>
+      <AdditionalDependencies>jvm.lib;ignite.common.lib;ignite.jni.lib;ignite.binary.lib;ignite.core.lib;%(AdditionalDependencies)</AdditionalDependencies>
+      <AdditionalLibraryDirectories>..\..\..\..\project\vs\$(Platform)\$(Configuration)\;$(JAVA_HOME)\lib;%(AdditionalLibraryDirectories)</AdditionalLibraryDirectories>
+    </Link>
+    <PostBuildEvent>
+      <Command>copy "$(ProjectDir)..\..\..\..\project\vs\$(Platform)\$(Configuration)\ignite.jni.dll" "$(OutDir)"
+copy "$(ProjectDir)..\..\..\..\project\vs\$(Platform)\$(Configuration)\ignite.core.dll" "$(OutDir)"</Command>
+    </PostBuildEvent>
+  </ItemDefinitionGroup>
+  <ItemGroup>
+    <ClCompile Include="..\..\src\put_get_example.cpp" />
+  </ItemGroup>
+  <ItemGroup>
+    <ClInclude Include="..\..\..\include\ignite\examples\address.h" />
+    <ClInclude Include="..\..\..\include\ignite\examples\organization.h" />
+  </ItemGroup>
+  <ItemGroup>
+    <None Include="..\..\config\example-cache.xml" />
+  </ItemGroup>
+  <Import Project="$(VCTargetsPath)\Microsoft.Cpp.targets" />
+  <ImportGroup Label="ExtensionTargets">
+  </ImportGroup>
+</Project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/a8586118/modules/platforms/cpp/examples/put-get-example/project/vs/put-get-example.vcxproj.filters
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/examples/put-get-example/project/vs/put-get-example.vcxproj.filters b/modules/platforms/cpp/examples/put-get-example/project/vs/put-get-example.vcxproj.filters
new file mode 100644
index 0000000..a50a431
--- /dev/null
+++ b/modules/platforms/cpp/examples/put-get-example/project/vs/put-get-example.vcxproj.filters
@@ -0,0 +1,38 @@
+<?xml version="1.0" encoding="utf-8"?>
+<Project ToolsVersion="4.0" xmlns="http://schemas.microsoft.com/developer/msbuild/2003">
+  <ItemGroup>
+    <Filter Include="Source Files">
+      <UniqueIdentifier>{4FC737F1-C7A5-4376-A066-2A32D752A2FF}</UniqueIdentifier>
+      <Extensions>cpp;c;cc;cxx;def;odl;idl;hpj;bat;asm;asmx</Extensions>
+    </Filter>
+    <Filter Include="Header Files">
+      <UniqueIdentifier>{93995380-89BD-4b04-88EB-625FBE52EBFB}</UniqueIdentifier>
+      <Extensions>h;hh;hpp;hxx;hm;inl;inc;xsd</Extensions>
+    </Filter>
+    <Filter Include="Resource Files">
+      <UniqueIdentifier>{67DA6AB6-F800-4c08-8B7A-83BB121AAD01}</UniqueIdentifier>
+      <Extensions>rc;ico;cur;bmp;dlg;rc2;rct;bin;rgs;gif;jpg;jpeg;jpe;resx;tiff;tif;png;wav;mfcribbon-ms</Extensions>
+    </Filter>
+    <Filter Include="Config">
+      <UniqueIdentifier>{487c5422-915c-4851-892d-c1599ea69e0c}</UniqueIdentifier>
+    </Filter>
+  </ItemGroup>
+  <ItemGroup>
+    <ClInclude Include="..\..\..\include\ignite\examples\address.h">
+      <Filter>Header Files</Filter>
+    </ClInclude>
+    <ClInclude Include="..\..\..\include\ignite\examples\organization.h">
+      <Filter>Header Files</Filter>
+    </ClInclude>
+  </ItemGroup>
+  <ItemGroup>
+    <None Include="..\..\config\example-cache.xml">
+      <Filter>Config</Filter>
+    </None>
+  </ItemGroup>
+  <ItemGroup>
+    <ClCompile Include="..\..\src\put_get_example.cpp">
+      <Filter>Source Files</Filter>
+    </ClCompile>
+  </ItemGroup>
+</Project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/a8586118/modules/platforms/cpp/examples/put-get-example/src/put_get_example.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/examples/put-get-example/src/put_get_example.cpp b/modules/platforms/cpp/examples/put-get-example/src/put_get_example.cpp
new file mode 100644
index 0000000..d8a401e
--- /dev/null
+++ b/modules/platforms/cpp/examples/put-get-example/src/put_get_example.cpp
@@ -0,0 +1,126 @@
+/*
+ * 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.
+ */
+
+#include <iostream>
+
+#include "ignite/ignite.h"
+#include "ignite/ignition.h"
+
+#include "ignite/examples/organization.h"
+
+using namespace ignite;
+using namespace cache;
+
+using namespace examples;
+
+/*
+ * Execute individual Put and Get operations.
+ * 
+ * @param cache Cache instance.
+ */
+void PutGet(Cache<int, Organization>& cache) 
+{
+    // Create new Organization to store in cache.
+    Organization org("Microsoft", Address("1096 Eddy Street, San Francisco, CA", 94109));
+
+    // Put organization to cache.
+    cache.Put(1, org);
+
+    // Get recently created employee as a strongly-typed fully de-serialized instance.
+    Organization orgFromCache = cache.Get(1);
+
+    std::cout <<  ">>> Retrieved organization instance from cache: " << std::endl;
+    std::cout << orgFromCache.ToString() << std::endl;
+    std::cout << std::endl;
+}
+
+/*
+ * Execute bulk Put and Get operations.
+ */
+void PutGetAll(Cache<int, Organization>& cache) 
+{
+    // Create new Organizations to store in cache.
+    Organization org1("Microsoft", Address("1096 Eddy Street, San Francisco, CA", 94109));
+    Organization org2("Red Cross", Address("184 Fidler Drive, San Antonio, TX", 78205));
+
+    // Put created data entries to cache.
+    std::map<int, Organization> vals;
+
+    vals[1] = org1;
+    vals[2] = org2;
+
+    cache.PutAll(vals);
+
+    // Get recently created organizations as a strongly-typed fully de-serialized instances.
+    std::set<int> keys;
+
+    keys.insert(1);
+    keys.insert(2);
+
+    std::map<int, Organization> valsFromCache = cache.GetAll(keys);
+
+    std::cout <<  ">>> Retrieved organization instances from cache: " << std::endl;
+
+    for (std::map<int, Organization>::iterator it = valsFromCache.begin(); it != valsFromCache.end(); ++it) 
+        std::cout <<  it->second.ToString() << std::endl;
+
+    std::cout << std::endl;
+}
+
+int main()
+{
+    IgniteConfiguration cfg;
+
+    cfg.jvmInitMem = 512;
+    cfg.jvmMaxMem = 512;
+
+    cfg.springCfgPath = "platforms/cpp/examples/put-get-example/config/example-cache.xml";
+
+    try
+    {
+        // Start a node.
+        Ignite grid = Ignition::Start(cfg);
+
+        std::cout << std::endl;
+        std::cout << ">>> Cache put-get example started." << std::endl;
+        std::cout << std::endl;
+
+        // Get cache instance.
+        Cache<int, Organization> cache = grid.GetCache<int, Organization>("atomic");
+
+        // Clear cache.
+        cache.Clear();
+
+        PutGet(cache);
+        PutGetAll(cache);
+
+        // Stop node.
+        Ignition::StopAll(false);
+    }
+    catch (IgniteError& err)
+    {
+        std::cout << "An error occurred: " << err.GetText() << std::endl;
+    }
+
+    std::cout << std::endl;
+    std::cout << ">>> Example finished, press 'Enter' to exit ..." << std::endl;
+    std::cout << std::endl;
+
+    std::cin.get();
+
+    return 0;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/a8586118/modules/platforms/cpp/examples/putget-example/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/examples/putget-example/Makefile.am b/modules/platforms/cpp/examples/putget-example/Makefile.am
deleted file mode 100644
index 88edc81..0000000
--- a/modules/platforms/cpp/examples/putget-example/Makefile.am
+++ /dev/null
@@ -1,58 +0,0 @@
-##
-## Licensed to the Apache Software Foundation (ASF) under one or more
-## contributor license agreements.  See the NOTICE file distributed with
-## this work for additional information regarding copyright ownership.
-## The ASF licenses this file to You under the Apache License, Version 2.0
-## (the "License"); you may not use this file except in compliance with
-## the License.  You may obtain a copy of the License at
-##
-##      http://www.apache.org/licenses/LICENSE-2.0
-##
-## Unless required by applicable law or agreed to in writing, software
-## distributed under the License is distributed on an "AS IS" BASIS,
-## WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-## See the License for the specific language governing permissions and
-## limitations under the License.
-##
-
-ACLOCAL_AMFLAGS =-I m4
-
-noinst_PROGRAMS = ignite-putgetexample
-
-AM_CPPFLAGS = \
-    -I@top_srcdir@/include \
-    -I@top_srcdir@/../core/include \
-    -I@top_srcdir@/../core/os/linux/include \
-    -I@top_srcdir@/../common/include \
-    -I@top_srcdir@/../common/os/linux/include \
-    -I@top_srcdir@/../binary/include \
-    -I@top_srcdir@/../jni/include \
-    -I@top_srcdir@/../jni/os/linux/include \
-    -I$(JAVA_HOME)/include \
-    -I$(JAVA_HOME)/include/linux \
-    -DIGNITE_IMPL \
-    -D__STDC_LIMIT_MACROS \
-    -D__STDC_CONSTANT_MACROS
-
-AM_CXXFLAGS = \
-    -Wall \
-    -std=c++03
-
-ignite_putgetexample_LDADD = \
-    @top_srcdir@/../core/libignite.la \
-    -lpthread
-
-ignite_putgetexample_LDFLAGS = \
-    -static-libtool-libs
-
-ignite_putgetexample_SOURCES = \
-    src/putget_example.cpp
-
-run-check: check
-	./ignite-putgetexample -p
-
-clean-local: clean-check
-	$(RM) *.gcno *.gcda
-
-clean-check:
-	$(RM) $(ignite_putgetexample_OBJECTS)

http://git-wip-us.apache.org/repos/asf/ignite/blob/a8586118/modules/platforms/cpp/examples/putget-example/config/example-cache.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/examples/putget-example/config/example-cache.xml b/modules/platforms/cpp/examples/putget-example/config/example-cache.xml
deleted file mode 100644
index af523cd..0000000
--- a/modules/platforms/cpp/examples/putget-example/config/example-cache.xml
+++ /dev/null
@@ -1,67 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-
-<!--
-  Licensed to the Apache Software Foundation (ASF) under one or more
-  contributor license agreements.  See the NOTICE file distributed with
-  this work for additional information regarding copyright ownership.
-  The ASF licenses this file to You under the Apache License, Version 2.0
-  (the "License"); you may not use this file except in compliance with
-  the License.  You may obtain a copy of the License at
-
-       http://www.apache.org/licenses/LICENSE-2.0
-
-  Unless required by applicable law or agreed to in writing, software
-  distributed under the License is distributed on an "AS IS" BASIS,
-  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  See the License for the specific language governing permissions and
-  limitations under the License.
--->
-
-<beans xmlns="http://www.springframework.org/schema/beans"
-       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-       xmlns:util="http://www.springframework.org/schema/util"
-       xsi:schemaLocation="
-        http://www.springframework.org/schema/beans
-        http://www.springframework.org/schema/beans/spring-beans.xsd
-        http://www.springframework.org/schema/util
-        http://www.springframework.org/schema/util/spring-util.xsd">
-    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
-        <!-- Set to true to enable distributed class loading for examples, default is false. -->
-        <property name="peerClassLoadingEnabled" value="true"/>
-
-        <property name="cacheConfiguration">
-            <list>
-                <!--
-                    Partitioned cache example configuration with binary objects enabled.
-                -->
-                <bean class="org.apache.ignite.configuration.CacheConfiguration">
-                    <property name="name" value="atomic"/>
-                    <property name="atomicityMode" value="ATOMIC"/>
-                    <property name="backups" value="1"/>
-                </bean>
-            </list>
-        </property>
-
-        <!-- Explicitly configure TCP discovery SPI to provide list of initial nodes. -->
-        <property name="discoverySpi">
-            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
-                <property name="ipFinder">
-                    <!--
-                        Ignite provides several options for automatic discovery that can be used
-                        instead os static IP based discovery.
-                    -->
-                    <!-- Uncomment static IP finder to enable static-based discovery of initial nodes. -->
-                    <!--<bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">-->
-                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.TcpDiscoveryMulticastIpFinder">
-                        <property name="addresses">
-                            <list>
-                                <!-- In distributed environment, replace with actual host IP address. -->
-                                <value>127.0.0.1:47500..47501</value>
-                            </list>
-                        </property>
-                    </bean>
-                </property>
-            </bean>
-        </property>
-    </bean>
-</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/a8586118/modules/platforms/cpp/examples/putget-example/project/vs/putget-example.vcxproj
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/examples/putget-example/project/vs/putget-example.vcxproj b/modules/platforms/cpp/examples/putget-example/project/vs/putget-example.vcxproj
deleted file mode 100644
index 8842f3a..0000000
--- a/modules/platforms/cpp/examples/putget-example/project/vs/putget-example.vcxproj
+++ /dev/null
@@ -1,110 +0,0 @@
-<?xml version="1.0" encoding="utf-8"?>
-<Project DefaultTargets="Build" ToolsVersion="4.0" xmlns="http://schemas.microsoft.com/developer/msbuild/2003">
-  <ItemGroup Label="ProjectConfigurations">
-    <ProjectConfiguration Include="Release|Win32">
-      <Configuration>Release</Configuration>
-      <Platform>Win32</Platform>
-    </ProjectConfiguration>
-    <ProjectConfiguration Include="Release|x64">
-      <Configuration>Release</Configuration>
-      <Platform>x64</Platform>
-    </ProjectConfiguration>
-  </ItemGroup>
-  <PropertyGroup Label="Globals">
-    <ProjectGuid>{34935DEC-80FC-4168-AA52-3DBFF4F79B6B}</ProjectGuid>
-    <Keyword>Win32Proj</Keyword>
-    <RootNamespace>igniteexamples</RootNamespace>
-  </PropertyGroup>
-  <Import Project="$(VCTargetsPath)\Microsoft.Cpp.Default.props" />
-  <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Release|x64'" Label="Configuration">
-    <ConfigurationType>Application</ConfigurationType>
-    <UseDebugLibraries>false</UseDebugLibraries>
-    <PlatformToolset>v100</PlatformToolset>
-    <WholeProgramOptimization>true</WholeProgramOptimization>
-    <CharacterSet>Unicode</CharacterSet>
-  </PropertyGroup>
-  <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Release|Win32'" Label="Configuration">
-    <ConfigurationType>Application</ConfigurationType>
-    <UseDebugLibraries>false</UseDebugLibraries>
-    <PlatformToolset>v100</PlatformToolset>
-    <WholeProgramOptimization>true</WholeProgramOptimization>
-    <CharacterSet>Unicode</CharacterSet>
-  </PropertyGroup>
-  <Import Project="$(VCTargetsPath)\Microsoft.Cpp.props" />
-  <ImportGroup Label="ExtensionSettings">
-  </ImportGroup>
-  <ImportGroup Condition="'$(Configuration)|$(Platform)'=='Release|x64'" Label="PropertySheets">
-    <Import Project="$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props" Condition="exists('$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props')" Label="LocalAppDataPlatform" />
-  </ImportGroup>
-  <ImportGroup Condition="'$(Configuration)|$(Platform)'=='Release|Win32'" Label="PropertySheets">
-    <Import Project="$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props" Condition="exists('$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props')" Label="LocalAppDataPlatform" />
-  </ImportGroup>
-  <PropertyGroup Label="UserMacros" />
-  <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Release|x64'">
-    <LinkIncremental>false</LinkIncremental>
-  </PropertyGroup>
-  <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Release|Win32'">
-    <LinkIncremental>false</LinkIncremental>
-  </PropertyGroup>
-  <ItemDefinitionGroup Condition="'$(Configuration)|$(Platform)'=='Release|x64'">
-    <ClCompile>
-      <WarningLevel>Level3</WarningLevel>
-      <PrecompiledHeader>
-      </PrecompiledHeader>
-      <Optimization>MaxSpeed</Optimization>
-      <FunctionLevelLinking>true</FunctionLevelLinking>
-      <IntrinsicFunctions>true</IntrinsicFunctions>
-      <PreprocessorDefinitions>WIN32;NDEBUG;_CONSOLE;_LIB;%(PreprocessorDefinitions)</PreprocessorDefinitions>
-      <AdditionalIncludeDirectories>$(JAVA_HOME)\include;$(JAVA_HOME)\include\win32;..\..\..\include;..\..\..\..\jni\os\win\include;..\..\..\..\jni\include;..\..\..\..\common\os\win\include;..\..\..\..\common\include;..\..\..\..\binary\include;..\..\..\..\core\os\win\include;..\..\..\..\core\include;%(AdditionalIncludeDirectories)</AdditionalIncludeDirectories>
-    </ClCompile>
-    <Link>
-      <SubSystem>Console</SubSystem>
-      <GenerateDebugInformation>true</GenerateDebugInformation>
-      <EnableCOMDATFolding>true</EnableCOMDATFolding>
-      <OptimizeReferences>true</OptimizeReferences>
-      <AdditionalDependencies>jvm.lib;ignite.common.lib;ignite.jni.lib;ignite.binary.lib;ignite.core.lib;%(AdditionalDependencies)</AdditionalDependencies>
-      <AdditionalLibraryDirectories>..\..\..\..\project\vs\$(Platform)\$(Configuration)\;$(JAVA_HOME)\lib;%(AdditionalLibraryDirectories)</AdditionalLibraryDirectories>
-    </Link>
-    <PostBuildEvent>
-      <Command>copy "$(ProjectDir)..\..\..\..\project\vs\$(Platform)\$(Configuration)\ignite.jni.dll" "$(OutDir)"
-copy "$(ProjectDir)..\..\..\..\project\vs\$(Platform)\$(Configuration)\ignite.core.dll" "$(OutDir)"</Command>
-    </PostBuildEvent>
-  </ItemDefinitionGroup>
-  <ItemDefinitionGroup Condition="'$(Configuration)|$(Platform)'=='Release|Win32'">
-    <ClCompile>
-      <WarningLevel>Level3</WarningLevel>
-      <PrecompiledHeader>
-      </PrecompiledHeader>
-      <Optimization>MaxSpeed</Optimization>
-      <FunctionLevelLinking>true</FunctionLevelLinking>
-      <IntrinsicFunctions>true</IntrinsicFunctions>
-      <PreprocessorDefinitions>WIN32;NDEBUG;_CONSOLE;_LIB;%(PreprocessorDefinitions)</PreprocessorDefinitions>
-      <AdditionalIncludeDirectories>$(JAVA_HOME)\include;$(JAVA_HOME)\include\win32;..\..\..\include;..\..\..\..\jni\os\win\include;..\..\..\..\jni\include;..\..\..\..\common\os\win\include;..\..\..\..\common\include;..\..\..\..\binary\include;..\..\..\..\core\os\win\include;..\..\..\..\core\include;%(AdditionalIncludeDirectories)</AdditionalIncludeDirectories>
-    </ClCompile>
-    <Link>
-      <SubSystem>Console</SubSystem>
-      <GenerateDebugInformation>true</GenerateDebugInformation>
-      <EnableCOMDATFolding>true</EnableCOMDATFolding>
-      <OptimizeReferences>true</OptimizeReferences>
-      <AdditionalDependencies>jvm.lib;ignite.common.lib;ignite.jni.lib;ignite.binary.lib;ignite.core.lib;%(AdditionalDependencies)</AdditionalDependencies>
-      <AdditionalLibraryDirectories>..\..\..\..\project\vs\$(Platform)\$(Configuration)\;$(JAVA_HOME)\lib;%(AdditionalLibraryDirectories)</AdditionalLibraryDirectories>
-    </Link>
-    <PostBuildEvent>
-      <Command>copy "$(ProjectDir)..\..\..\..\project\vs\$(Platform)\$(Configuration)\ignite.jni.dll" "$(OutDir)"
-copy "$(ProjectDir)..\..\..\..\project\vs\$(Platform)\$(Configuration)\ignite.core.dll" "$(OutDir)"</Command>
-    </PostBuildEvent>
-  </ItemDefinitionGroup>
-  <ItemGroup>
-    <ClCompile Include="..\..\src\putget_example.cpp" />
-  </ItemGroup>
-  <ItemGroup>
-    <ClInclude Include="..\..\..\include\ignite\examples\address.h" />
-    <ClInclude Include="..\..\..\include\ignite\examples\organization.h" />
-  </ItemGroup>
-  <ItemGroup>
-    <None Include="..\..\config\example-cache.xml" />
-  </ItemGroup>
-  <Import Project="$(VCTargetsPath)\Microsoft.Cpp.targets" />
-  <ImportGroup Label="ExtensionTargets">
-  </ImportGroup>
-</Project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/a8586118/modules/platforms/cpp/examples/putget-example/project/vs/putget-example.vcxproj.filters
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/examples/putget-example/project/vs/putget-example.vcxproj.filters b/modules/platforms/cpp/examples/putget-example/project/vs/putget-example.vcxproj.filters
deleted file mode 100644
index 3bb8a8f..0000000
--- a/modules/platforms/cpp/examples/putget-example/project/vs/putget-example.vcxproj.filters
+++ /dev/null
@@ -1,38 +0,0 @@
-<?xml version="1.0" encoding="utf-8"?>
-<Project ToolsVersion="4.0" xmlns="http://schemas.microsoft.com/developer/msbuild/2003">
-  <ItemGroup>
-    <Filter Include="Source Files">
-      <UniqueIdentifier>{4FC737F1-C7A5-4376-A066-2A32D752A2FF}</UniqueIdentifier>
-      <Extensions>cpp;c;cc;cxx;def;odl;idl;hpj;bat;asm;asmx</Extensions>
-    </Filter>
-    <Filter Include="Header Files">
-      <UniqueIdentifier>{93995380-89BD-4b04-88EB-625FBE52EBFB}</UniqueIdentifier>
-      <Extensions>h;hh;hpp;hxx;hm;inl;inc;xsd</Extensions>
-    </Filter>
-    <Filter Include="Resource Files">
-      <UniqueIdentifier>{67DA6AB6-F800-4c08-8B7A-83BB121AAD01}</UniqueIdentifier>
-      <Extensions>rc;ico;cur;bmp;dlg;rc2;rct;bin;rgs;gif;jpg;jpeg;jpe;resx;tiff;tif;png;wav;mfcribbon-ms</Extensions>
-    </Filter>
-    <Filter Include="Config">
-      <UniqueIdentifier>{487c5422-915c-4851-892d-c1599ea69e0c}</UniqueIdentifier>
-    </Filter>
-  </ItemGroup>
-  <ItemGroup>
-    <ClCompile Include="..\..\src\putget_example.cpp">
-      <Filter>Source Files</Filter>
-    </ClCompile>
-  </ItemGroup>
-  <ItemGroup>
-    <ClInclude Include="..\..\..\include\ignite\examples\address.h">
-      <Filter>Header Files</Filter>
-    </ClInclude>
-    <ClInclude Include="..\..\..\include\ignite\examples\organization.h">
-      <Filter>Header Files</Filter>
-    </ClInclude>
-  </ItemGroup>
-  <ItemGroup>
-    <None Include="..\..\config\example-cache.xml">
-      <Filter>Config</Filter>
-    </None>
-  </ItemGroup>
-</Project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/a8586118/modules/platforms/cpp/examples/putget-example/src/putget_example.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/examples/putget-example/src/putget_example.cpp b/modules/platforms/cpp/examples/putget-example/src/putget_example.cpp
deleted file mode 100644
index f7bd894..0000000
--- a/modules/platforms/cpp/examples/putget-example/src/putget_example.cpp
+++ /dev/null
@@ -1,126 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-#include <iostream>
-
-#include "ignite/ignite.h"
-#include "ignite/ignition.h"
-
-#include "ignite/examples/organization.h"
-
-using namespace ignite;
-using namespace cache;
-
-using namespace examples;
-
-/*
- * Execute individual Put and Get operations.
- * 
- * @param cache Cache instance.
- */
-void PutGet(Cache<int, Organization>& cache) 
-{
-    // Create new Organization to store in cache.
-    Organization org("Microsoft", Address("1096 Eddy Street, San Francisco, CA", 94109));
-
-    // Put organization to cache.
-    cache.Put(1, org);
-
-    // Get recently created employee as a strongly-typed fully de-serialized instance.
-    Organization orgFromCache = cache.Get(1);
-
-    std::cout <<  ">>> Retrieved organization instance from cache: " << std::endl;
-    std::cout << orgFromCache.ToString() << std::endl;
-    std::cout << std::endl;
-}
-
-/*
- * Execute bulk Put and Get operations.
- */
-void PutGetAll(Cache<int, Organization>& cache) 
-{
-    // Create new Organizations to store in cache.
-    Organization org1("Microsoft", Address("1096 Eddy Street, San Francisco, CA", 94109));
-    Organization org2("Red Cross", Address("184 Fidler Drive, San Antonio, TX", 78205));
-
-    // Put created data entries to cache.
-    std::map<int, Organization> vals;
-
-    vals[1] = org1;
-    vals[2] = org2;
-
-    cache.PutAll(vals);
-
-    // Get recently created organizations as a strongly-typed fully de-serialized instances.
-    std::set<int> keys;
-
-    keys.insert(1);
-    keys.insert(2);
-
-    std::map<int, Organization> valsFromCache = cache.GetAll(keys);
-
-    std::cout <<  ">>> Retrieved organization instances from cache: " << std::endl;
-
-    for (std::map<int, Organization>::iterator it = valsFromCache.begin(); it != valsFromCache.end(); ++it) 
-        std::cout <<  it->second.ToString() << std::endl;
-
-    std::cout << std::endl;
-}
-
-int main()
-{
-    IgniteConfiguration cfg;
-
-    cfg.jvmInitMem = 512;
-    cfg.jvmMaxMem = 512;
-
-    cfg.springCfgPath = "platforms/cpp/examples/putget-example/config/example-cache.xml";
-
-    try
-    {
-        // Start a node.
-        Ignite grid = Ignition::Start(cfg);
-
-        std::cout << std::endl;
-        std::cout << ">>> Cache put-get example started." << std::endl;
-        std::cout << std::endl;
-
-        // Get cache instance.
-        Cache<int, Organization> cache = grid.GetCache<int, Organization>("atomic");
-
-        // Clear cache.
-        cache.Clear();
-
-        PutGet(cache);
-        PutGetAll(cache);
-
-        // Stop node.
-        Ignition::StopAll(false);
-    }
-    catch (IgniteError& err)
-    {
-        std::cout << "An error occurred: " << err.GetText() << std::endl;
-    }
-
-    std::cout << std::endl;
-    std::cout << ">>> Example finished, press 'Enter' to exit ..." << std::endl;
-    std::cout << std::endl;
-
-    std::cin.get();
-
-    return 0;
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/a8586118/modules/platforms/cpp/examples/query-example/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/examples/query-example/Makefile.am b/modules/platforms/cpp/examples/query-example/Makefile.am
index 1d00bcf..368d4bc 100644
--- a/modules/platforms/cpp/examples/query-example/Makefile.am
+++ b/modules/platforms/cpp/examples/query-example/Makefile.am
@@ -17,7 +17,7 @@
 
 ACLOCAL_AMFLAGS =-I m4
 
-noinst_PROGRAMS = ignite-queryexample
+noinst_PROGRAMS = ignite-query-example
 
 AM_CPPFLAGS = \
     -I@top_srcdir@/include \
@@ -38,21 +38,21 @@ AM_CXXFLAGS = \
     -Wall \
     -std=c++03
 
-ignite_queryexample_LDADD = \
+ignite_query_example_LDADD = \
     @top_srcdir@/../core/libignite.la \
     -lpthread
 
-ignite_queryexample_LDFLAGS = \
+ignite_query_example_LDFLAGS = \
     -static-libtool-libs
 
-ignite_queryexample_SOURCES = \
+ignite_query_example_SOURCES = \
     src/query_example.cpp
 
 run-check: check
-	./ignite-queryexample -p
+	./ignite-query-example -p
 
 clean-local: clean-check
 	$(RM) *.gcno *.gcda
 
 clean-check:
-	$(RM) $(ignite_queryexample_OBJECTS)
+	$(RM) $(ignite_query_example_OBJECTS)


[19/19] ignite git commit: Merge remote-tracking branch 'remotes/origin/master' into ignite-5075

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/master' into ignite-5075

# Conflicts:
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java


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

Branch: refs/heads/ignite-5075
Commit: 08ed6da988a65c76d5d9e900448e7aa0bff1de78
Parents: b74b739
Author: sboikov <sb...@gridgain.com>
Authored: Wed May 31 10:35:22 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed May 31 10:35:22 2017 +0300

----------------------------------------------------------------------
 .../ignite/internal/processors/cache/ClusterCachesInfo.java  | 8 ++++----
 .../internal/processors/cache/GridCacheAttributes.java       | 7 ++++---
 2 files changed, 8 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/08ed6da9/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
index 1978f9e..cd79673 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
@@ -143,8 +143,8 @@ class ClusterCachesInfo {
         CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "cacheMode", "Cache mode",
             locAttr.cacheMode(), rmtAttr.cacheMode(), true);
 
-        CU.checkAttributeMismatch(log, rmtCfg.getGroupName(), rmt, "groupName", "Cache group name",
-            locCfg.getGroupName(), rmtCfg.getGroupName(), true);
+        CU.checkAttributeMismatch(log, rmtAttr.groupName(), rmt, "groupName", "Cache group name",
+            locAttr.groupName(), rmtAttr.groupName(), true);
 
         CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "sql", "SQL flag",
             locAttr.sql(), rmtAttr.sql(), true);
@@ -1089,8 +1089,8 @@ class ClusterCachesInfo {
      */
     private void validateCacheGroupConfiguration(CacheConfiguration cfg, CacheConfiguration startCfg)
         throws IgniteCheckedException {
-        GridCacheAttributes attr1 = new GridCacheAttributes(cfg);
-        GridCacheAttributes attr2 = new GridCacheAttributes(startCfg);
+        GridCacheAttributes attr1 = new GridCacheAttributes(cfg, false);
+        GridCacheAttributes attr2 = new GridCacheAttributes(startCfg, false);
 
         CU.validateCacheGroupsAttributesMismatch(log, cfg, startCfg, "cacheMode", "Cache mode",
             cfg.getCacheMode(), startCfg.getCacheMode(), true);

http://git-wip-us.apache.org/repos/asf/ignite/blob/08ed6da9/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
index ed695dc..7d97159 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
@@ -52,6 +52,7 @@ public class GridCacheAttributes implements Serializable {
 
     /**
      * @param cfg Cache configuration.
+     * @param sql SQL flag.
      */
     public GridCacheAttributes(CacheConfiguration cfg, boolean sql) {
         ccfg = cfg;
@@ -60,10 +61,10 @@ public class GridCacheAttributes implements Serializable {
     }
 
     /**
-     * Public no-arg constructor for {@link Externalizable}.
+     * @return Cache group name.
      */
-    public GridCacheAttributes() {
-        // No-op.
+    public String groupName() {
+        return ccfg.getGroupName();
     }
 
     /**


[09/19] ignite git commit: .NET: Fix compute exception propagation: throw AggregateException unchanged

Posted by sb...@apache.org.
.NET: Fix compute exception propagation: throw AggregateException unchanged


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

Branch: refs/heads/ignite-5075
Commit: 618a448ab5fa2075a212b6a2abe4719a010038c3
Parents: 03b383c
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Mon May 29 23:37:47 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Mon May 29 23:37:47 2017 +0300

----------------------------------------------------------------------
 .../Compute/BinarizableClosureTaskTest.cs               |  7 ++-----
 .../Apache.Ignite.Core.Tests/Compute/ClosureTaskTest.cs |  6 ++++--
 .../Apache.Ignite.Core.Tests/Compute/ComputeApiTest.cs  | 10 ++++------
 .../Compute/IgniteExceptionTaskSelfTest.cs              |  8 ++++++--
 .../Compute/SerializableClosureTaskTest.cs              |  5 +----
 .../Deployment/PeerAssemblyLoadingTest.cs               |  5 +++--
 .../Apache.Ignite.Core.Tests/Log/CustomLoggerTest.cs    |  5 +++--
 .../Impl/Binary/BinaryReflectiveActions.cs              |  7 +++++--
 .../dotnet/Apache.Ignite.Core/Impl/Common/Future.cs     | 12 ++----------
 9 files changed, 30 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/618a448a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/BinarizableClosureTaskTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/BinarizableClosureTaskTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/BinarizableClosureTaskTest.cs
index e8952bf..3482456 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/BinarizableClosureTaskTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/BinarizableClosureTaskTest.cs
@@ -19,6 +19,7 @@ namespace Apache.Ignite.Core.Tests.Compute
 {
     using System;
     using Apache.Ignite.Core.Binary;
+    using Apache.Ignite.Core.Common;
     using Apache.Ignite.Core.Compute;
     using NUnit.Framework;
 
@@ -67,11 +68,7 @@ namespace Apache.Ignite.Core.Tests.Compute
         {
             Assert.IsTrue(err != null);
 
-            var aggregate = err as AggregateException;
-
-            if (aggregate != null)
-                err = aggregate.InnerException;
-
+            err = err.InnerException;
             Assert.IsNotNull(err);
 
             var err0 = err.InnerException as BinarizableException;

http://git-wip-us.apache.org/repos/asf/ignite/blob/618a448a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ClosureTaskTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ClosureTaskTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ClosureTaskTest.cs
index ffb2844..a3b87ca 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ClosureTaskTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ClosureTaskTest.cs
@@ -213,9 +213,11 @@ namespace Apache.Ignite.Core.Tests.Compute
         {
             var args = Enumerable.Repeat(1, MultiCloCnt).Cast<object>().ToArray();
 
-            var e = Assert.Throws<Exception>(() => Grid1.GetCompute().Apply(Func(false), args, new Reducer(true)));
+            var e = Assert.Throws<AggregateException>(() =>
+                Grid1.GetCompute().Apply(Func(false), args, new Reducer(true)));
 
-            Assert.AreEqual(ErrMsg, e.Message);
+            Assert.IsNotNull(e.InnerException);
+            Assert.AreEqual(ErrMsg, e.InnerException.Message);
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/618a448a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTest.cs
index e4fd853..0c6d20e 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTest.cs
@@ -1285,17 +1285,16 @@ namespace Apache.Ignite.Core.Tests.Compute
         [Test]
         public void TestExceptions()
         {
-            Assert.Throws<IgniteException>(() => _grid1.GetCompute().Broadcast(new InvalidComputeAction()));
+            Assert.Throws<AggregateException>(() => _grid1.GetCompute().Broadcast(new InvalidComputeAction()));
 
-            Assert.Throws<IgniteException>(
+            Assert.Throws<AggregateException>(
                 () => _grid1.GetCompute().Execute<NetSimpleJobArgument, NetSimpleJobResult, NetSimpleTaskResult>(
                     typeof (NetSimpleTask), new NetSimpleJobArgument(-1)));
 
             // Local.
-            var ex = Assert.Throws<IgniteException>(() =>
+            var ex = Assert.Throws<AggregateException>(() =>
                 _grid1.GetCluster().ForLocal().GetCompute().Broadcast(new ExceptionalComputeAction()));
 
-            Assert.AreEqual("Async operation has failed, examine InnerException for details.", ex.Message);
             Assert.IsNotNull(ex.InnerException);
             Assert.AreEqual("Compute job has failed on local node, examine InnerException for details.", 
                 ex.InnerException.Message);
@@ -1303,10 +1302,9 @@ namespace Apache.Ignite.Core.Tests.Compute
             Assert.AreEqual(ExceptionalComputeAction.ErrorText, ex.InnerException.InnerException.Message);
 
             // Remote.
-            ex = Assert.Throws<IgniteException>(() =>
+            ex = Assert.Throws<AggregateException>(() =>
                 _grid1.GetCluster().ForRemotes().GetCompute().Broadcast(new ExceptionalComputeAction()));
 
-            Assert.AreEqual("Async operation has failed, examine InnerException for details.", ex.Message);
             Assert.IsNotNull(ex.InnerException);
             Assert.AreEqual("Compute job has failed on remote node, examine InnerException for details.",
                 ex.InnerException.Message);

http://git-wip-us.apache.org/repos/asf/ignite/blob/618a448a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/IgniteExceptionTaskSelfTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/IgniteExceptionTaskSelfTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/IgniteExceptionTaskSelfTest.cs
index 2f9f6b4..9f157f4 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/IgniteExceptionTaskSelfTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/IgniteExceptionTaskSelfTest.cs
@@ -168,7 +168,8 @@ namespace Apache.Ignite.Core.Tests.Compute
         {
             _mode = ErrorMode.RmtJobErrNotMarshalable;
 
-            Assert.Throws<SerializationException>(() => Execute());
+            var ex = Assert.Throws<AggregateException>(() => Execute());
+            Assert.IsInstanceOf<SerializationException>(ex.InnerException);
         }
 
         /// <summary>
@@ -322,7 +323,10 @@ namespace Apache.Ignite.Core.Tests.Compute
         {
             JobErrs.Clear();
 
-            return Assert.Catch(() => Grid1.GetCompute().Execute(new Task()));
+            var ex = Assert.Throws<AggregateException>(() => Grid1.GetCompute().Execute(new Task()));
+
+            Assert.IsNotNull(ex.InnerException);
+            return ex.InnerException;
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/618a448a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/SerializableClosureTaskTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/SerializableClosureTaskTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/SerializableClosureTaskTest.cs
index ecb4aa5..48d544f 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/SerializableClosureTaskTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/SerializableClosureTaskTest.cs
@@ -67,10 +67,7 @@ namespace Apache.Ignite.Core.Tests.Compute
         {
             Assert.IsTrue(err != null);
 
-            var aggregate = err as AggregateException;
-
-            if (aggregate != null)
-                err = aggregate.InnerException;
+            err = err.InnerException;
 
             Assert.IsNotNull(err);
             SerializableException err0 = err.InnerException as SerializableException;

http://git-wip-us.apache.org/repos/asf/ignite/blob/618a448a/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
index c74375d..de1ecd6 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Deployment/PeerAssemblyLoadingTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Deployment/PeerAssemblyLoadingTest.cs
@@ -22,7 +22,6 @@ namespace Apache.Ignite.Core.Tests.Deployment
     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;
@@ -47,8 +46,10 @@ namespace Apache.Ignite.Core.Tests.Deployment
         {
             TestDeployment(remoteCompute =>
             {
-                var ex = Assert.Throws<IgniteException>(() => remoteCompute.Call(new ProcessNameFunc()));
+                var ex = Assert.Throws<AggregateException>(() => remoteCompute.Call(new ProcessNameFunc()))
+                    .InnerException;
 
+                Assert.IsNotNull(ex);
                 Assert.AreEqual("Compute job has failed on remote node, examine InnerException for details.", 
                     ex.Message);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/618a448a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Log/CustomLoggerTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Log/CustomLoggerTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Log/CustomLoggerTest.cs
index 7d4c945..cb70f1e 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Log/CustomLoggerTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Log/CustomLoggerTest.cs
@@ -130,8 +130,9 @@ namespace Apache.Ignite.Core.Tests.Log
             {
                 var compute = ignite.GetCluster().ForRemotes().GetCompute();
 
-                var ex = Assert.Throws<IgniteException>(() => compute.Call(new FailFunc()));
-                Assert.IsInstanceOf<ArithmeticException>(ex.InnerException);
+                var ex = Assert.Throws<AggregateException>(() => compute.Call(new FailFunc()));
+                Assert.IsNotNull(ex.InnerException);
+                Assert.IsInstanceOf<ArithmeticException>(ex.InnerException.InnerException);
 
                 // Log updates may not arrive immediately
                 TestUtils.WaitForCondition(() => TestLogger.Entries.Any(x => x.Exception != null), 3000);

http://git-wip-us.apache.org/repos/asf/ignite/blob/618a448a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReflectiveActions.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReflectiveActions.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReflectiveActions.cs
index 5b6e5f1..b061be2 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReflectiveActions.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReflectiveActions.cs
@@ -226,7 +226,7 @@ namespace Apache.Ignite.Core.Impl.Binary
                     : GetWriter<float>(field, (f, w, o) => w.WriteFloat(f, o));
                 readAction = raw ? GetRawReader(field, r => r.ReadFloat()) : GetReader(field, (f, r) => r.ReadFloat(f));
             }
-            else if (type == typeof (double))
+            else if (type == typeof(double))
             {
                 writeAction = raw
                     ? GetRawWriter<double>(field, (w, o) => w.WriteDouble(o))
@@ -236,7 +236,10 @@ namespace Apache.Ignite.Core.Impl.Binary
                     : GetReader(field, (f, r) => r.ReadDouble(f));
             }
             else
-                throw new IgniteException("Unsupported primitive type: " + type.Name);
+            {
+                throw new IgniteException(string.Format("Unsupported primitive type '{0}' [Field={1}, " +
+                                                        "DeclaringType={2}", type, field, field.DeclaringType));
+            }
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/618a448a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/Future.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/Future.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/Future.cs
index b69ad56..314e531 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/Future.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/Future.cs
@@ -52,18 +52,10 @@ namespace Apache.Ignite.Core.Impl.Common
         /// <summary>
         /// Gets the result.
         /// </summary>
+        /// <exception cref="AggregateException" />
         public T Get()
         {
-            try
-            {
-                return Task.Result;
-            }
-            catch (AggregateException ex)
-            {
-                var innerEx = ex.InnerExceptions.Count > 1 ? ex : ex.InnerException;
-
-                throw new IgniteException("Async operation has failed, examine InnerException for details.", innerEx);
-            }
+            return Task.Result;
         }
 
         /// <summary>


[06/19] ignite git commit: .NET: Fix code analysis warnings

Posted by sb...@apache.org.
.NET: Fix code analysis warnings


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

Branch: refs/heads/ignite-5075
Commit: cee5dd84a23cf0456e30149767a7cf5ba70841f6
Parents: f9c96de
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Mon May 29 20:01:46 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Mon May 29 20:01:46 2017 +0300

----------------------------------------------------------------------
 .../dotnet/Apache.Ignite.Core/Cache/Configuration/QueryEntity.cs | 4 ++--
 modules/platforms/dotnet/Apache.Ignite.FxCop                     | 3 +--
 2 files changed, 3 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/cee5dd84/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryEntity.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryEntity.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryEntity.cs
index 58e5af9..ff61394 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryEntity.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryEntity.cs
@@ -34,7 +34,7 @@ namespace Apache.Ignite.Core.Cache.Configuration
     /// Query entity is a description of cache entry (composed of key and value) 
     /// in a way of how it must be indexed and can be queried.
     /// </summary>
-    public class QueryEntity : IQueryEntityInternal
+    public sealed class QueryEntity : IQueryEntityInternal
     {
         /** */
         private Type _keyType;
@@ -216,7 +216,7 @@ namespace Apache.Ignite.Core.Cache.Configuration
             // PERF: No ToDictionary.
             if (_aliasMap == null)
             {
-                _aliasMap = new Dictionary<string, string>(Aliases.Count, StringComparer.InvariantCulture);
+                _aliasMap = new Dictionary<string, string>(Aliases.Count, StringComparer.Ordinal);
 
                 foreach (var alias in Aliases)
                 {

http://git-wip-us.apache.org/repos/asf/ignite/blob/cee5dd84/modules/platforms/dotnet/Apache.Ignite.FxCop
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.FxCop b/modules/platforms/dotnet/Apache.Ignite.FxCop
index 109f59f..3653fa0 100644
--- a/modules/platforms/dotnet/Apache.Ignite.FxCop
+++ b/modules/platforms/dotnet/Apache.Ignite.FxCop
@@ -21,9 +21,9 @@
   <IgnoreGeneratedCode>False</IgnoreGeneratedCode>
  </ProjectOptions>
  <Targets>
+  <Target Name="$(ProjectDir)/Apache.Ignite.AspNet/bin/Debug/Apache.Ignite.AspNet.dll" Analyze="True" AnalyzeAllChildren="True" />
   <Target Name="$(ProjectDir)/Apache.Ignite.Core/bin/Debug/Apache.Ignite.Core.dll" Analyze="True" AnalyzeAllChildren="True" />
   <Target Name="$(ProjectDir)/Apache.Ignite.Linq/bin/Debug/Apache.Ignite.Linq.dll" Analyze="True" AnalyzeAllChildren="True" />
-  <Target Name="$(ProjectDir)/Apache.Ignite.AspNet/bin/Debug/Apache.Ignite.AspNet.dll" Analyze="True" AnalyzeAllChildren="True" />
   <Target Name="$(ProjectDir)/Apache.Ignite.NLog/bin/Debug/Apache.Ignite.NLog.dll" Analyze="True" AnalyzeAllChildren="True" />
  </Targets>
  <Rules>
@@ -33,7 +33,6 @@
     <Rule Name="AssembliesShouldHaveValidStrongNames" Enabled="True" />
     <Rule Name="AvoidEmptyInterfaces" Enabled="True" />
     <Rule Name="CollectionsShouldImplementGenericInterface" Enabled="True" />
-    <Rule Name="ConsiderPassingBaseTypesAsParameters" Enabled="True" />
     <Rule Name="DeclareEventHandlersCorrectly" Enabled="True" />
     <Rule Name="DeclareTypesInNamespaces" Enabled="True" />
     <Rule Name="DefineAccessorsForAttributeArguments" Enabled="True" />


[04/19] ignite git commit: IGNITE-3355: Implemented Compute::Call() for C++

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/f9c96de5/modules/platforms/cpp/core/include/ignite/impl/compute/compute_job_holder.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/impl/compute/compute_job_holder.h b/modules/platforms/cpp/core/include/ignite/impl/compute/compute_job_holder.h
new file mode 100644
index 0000000..e218e36
--- /dev/null
+++ b/modules/platforms/cpp/core/include/ignite/impl/compute/compute_job_holder.h
@@ -0,0 +1,139 @@
+/*
+ * 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.
+ */
+
+/**
+ * @file
+ * Declares ignite::impl::compute::ComputeJobHolder class template.
+ */
+
+#ifndef _IGNITE_IMPL_COMPUTE_COMPUTE_JOB_HOLDER
+#define _IGNITE_IMPL_COMPUTE_COMPUTE_JOB_HOLDER
+
+#include <ignite/impl/binary/binary_writer_impl.h>
+#include <ignite/impl/compute/compute_job_result.h>
+
+namespace ignite
+{
+    namespace impl
+    {
+        namespace compute
+        {
+            /**
+             * Compute job holder. Internal helper class.
+             * Used to handle jobs in general way, without specific types.
+             */
+            class ComputeJobHolder
+            {
+            public:
+                /**
+                 * Destructor.
+                 */
+                virtual ~ComputeJobHolder()
+                {
+                    // No-op.
+                }
+
+                /**
+                 * Execute job locally.
+                 */
+                virtual void ExecuteLocal() = 0;
+
+                /**
+                 * Execute job remote.
+                 *
+                 * @param writer Writer.
+                 */
+                virtual void ExecuteRemote(binary::BinaryWriterImpl& writer) = 0;
+            };
+
+            /**
+             * Compute job holder. Internal class.
+             *
+             * @tparam F Actual job type.
+             * @tparam R Job return type.
+             */
+            template<typename F, typename R>
+            class ComputeJobHolderImpl : public ComputeJobHolder
+            {
+            public:
+                typedef R ResultType;
+                typedef F JobType;
+
+                /**
+                 * Constructor.
+                 *
+                 * @param job Job.
+                 */
+                ComputeJobHolderImpl(JobType job) :
+                    job(job)
+                {
+                    // No-op.
+                }
+
+                /**
+                 * Destructor.
+                 */
+                virtual ~ComputeJobHolderImpl()
+                {
+                    // No-op.
+                }
+
+                const ComputeJobResult<ResultType>& GetResult()
+                {
+                    return res;
+                }
+
+                virtual void ExecuteLocal()
+                {
+                    try
+                    {
+                        res.SetResult(job.Call());
+                    }
+                    catch (const IgniteError& err)
+                    {
+                        res.SetError(err);
+                    }
+                    catch (const std::exception& err)
+                    {
+                        res.SetError(IgniteError(IgniteError::IGNITE_ERR_STD, err.what()));
+                    }
+                    catch (...)
+                    {
+                        res.SetError(IgniteError(IgniteError::IGNITE_ERR_UNKNOWN,
+                            "Unknown error occurred during call."));
+                    }
+                }
+
+                virtual void ExecuteRemote(binary::BinaryWriterImpl& writer)
+                {
+                    ExecuteLocal();
+
+                    res.Write(writer);
+                }
+
+            private:
+                /** Result. */
+                ComputeJobResult<ResultType> res;
+
+                /** Job. */
+                JobType job;
+            };
+        }
+    }
+}
+
+#endif //_IGNITE_IMPL_COMPUTE_COMPUTE_JOB_HOLDER

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9c96de5/modules/platforms/cpp/core/include/ignite/impl/compute/compute_job_result.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/impl/compute/compute_job_result.h b/modules/platforms/cpp/core/include/ignite/impl/compute/compute_job_result.h
new file mode 100644
index 0000000..5bcb762
--- /dev/null
+++ b/modules/platforms/cpp/core/include/ignite/impl/compute/compute_job_result.h
@@ -0,0 +1,161 @@
+/*
+ * 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.
+ */
+
+/**
+ * @file
+ * Declares ignite::impl::compute::ComputeJobResult class template.
+ */
+
+#ifndef _IGNITE_IMPL_COMPUTE_COMPUTE_JOB_RESULT
+#define _IGNITE_IMPL_COMPUTE_COMPUTE_JOB_RESULT
+
+#include <memory>
+#include <sstream>
+
+#include <ignite/common/promise.h>
+
+namespace ignite
+{
+    namespace impl
+    {
+        namespace compute
+        {
+            /**
+             * Used to hold compute job result.
+             */
+            template<typename R>
+            class ComputeJobResult
+            {
+            public:
+                typedef R ResultType;
+                /**
+                 * Default constructor.
+                 */
+                ComputeJobResult() :
+                    res(),
+                    err()
+                {
+                    // No-op.
+                }
+
+                /**
+                 * Set result value.
+                 *
+                 * @param val Value to set as a result.
+                 */
+                void SetResult(const ResultType& val)
+                {
+                    res = val;
+                }
+
+                /**
+                 * Set error.
+                 *
+                 * @param error Error to set.
+                 */
+                void SetError(const IgniteError error)
+                {
+                    err = error;
+                }
+
+                /**
+                 * Set promise to a state which corresponds to result.
+                 *
+                 * @param promise Promise, which state to set.
+                 */
+                void SetPromise(common::Promise<ResultType>& promise)
+                {
+                    if (err.GetCode() != IgniteError::IGNITE_SUCCESS)
+                        promise.SetError(err);
+                    else
+                        promise.SetValue(std::auto_ptr<ResultType>(new ResultType(res)));
+                }
+
+                /**
+                 * Write using writer.
+                 *
+                 * @param writer Writer.
+                 */
+                void Write(binary::BinaryWriterImpl& writer)
+                {
+                    if (err.GetCode() != IgniteError::IGNITE_SUCCESS)
+                    {
+                        // Fail
+                        writer.WriteBool(false);
+
+                        // Native Exception
+                        writer.WriteBool(true);
+
+                        writer.WriteObject<IgniteError>(err);
+                    }
+                    else
+                    {
+                        // Success
+                        writer.WriteBool(true);
+
+                        writer.WriteObject<ResultType>(res);
+                    }
+                }
+
+                /**
+                 * Read using reader.
+                 *
+                 * @param reader Reader.
+                 */
+                void Read(binary::BinaryReaderImpl& reader)
+                {
+                    bool success = reader.ReadBool();
+
+                    if (success)
+                    {
+                        res = reader.ReadObject<ResultType>();
+
+                        err = IgniteError();
+                    }
+                    else
+                    {
+                        bool native = reader.ReadBool();
+
+                        if (native)
+                            err = reader.ReadObject<IgniteError>();
+                        else
+                        {
+                            std::stringstream buf;
+
+                            buf << reader.ReadObject<std::string>() << " : ";
+                            buf << reader.ReadObject<std::string>() << ", ";
+                            buf << reader.ReadObject<std::string>();
+
+                            std::string msg = buf.str();
+
+                            err = IgniteError(IgniteError::IGNITE_ERR_GENERIC, msg.c_str());
+                        }
+                    }
+                }
+
+            private:
+                /** Result. */
+                ResultType res;
+
+                /** Erorr. */
+                IgniteError err;
+            };
+        }
+    }
+}
+
+#endif //_IGNITE_IMPL_COMPUTE_COMPUTE_JOB_RESULT

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9c96de5/modules/platforms/cpp/core/include/ignite/impl/compute/compute_task_holder.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/impl/compute/compute_task_holder.h b/modules/platforms/cpp/core/include/ignite/impl/compute/compute_task_holder.h
new file mode 100644
index 0000000..bdd7513
--- /dev/null
+++ b/modules/platforms/cpp/core/include/ignite/impl/compute/compute_task_holder.h
@@ -0,0 +1,213 @@
+/*
+ * 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.
+ */
+
+/**
+ * @file
+ * Declares ignite::impl::compute::ComputeTaskHolder class and
+ * ignite::impl::compute::ComputeTaskHolderImpl class template.
+ */
+
+#ifndef _IGNITE_IMPL_COMPUTE_COMPUTE_TASK_IMPL
+#define _IGNITE_IMPL_COMPUTE_COMPUTE_TASK_IMPL
+
+#include <stdint.h>
+
+#include <ignite/common/promise.h>
+#include <ignite/impl/compute/compute_job_result.h>
+#include <ignite/impl/compute/compute_job_holder.h>
+
+namespace ignite
+{
+    namespace impl
+    {
+        namespace compute
+        {
+            struct ComputeJobResultPolicy
+            {
+                enum Type
+                {
+                    /**
+                     * Wait for results if any are still expected. If all results have been received -
+                     * it will start reducing results.
+                     */
+                    WAIT = 0,
+
+                    /**
+                     * Ignore all not yet received results and start reducing results.
+                     */
+                    REDUCE = 1,
+
+                    /**
+                     * Fail-over job to execute on another node.
+                     */
+                    FAILOVER = 2
+                };
+            };
+
+            /**
+             * Compute task holder. Internal helper class.
+             * Used to handle tasks in general way, without specific types.
+             */
+            class ComputeTaskHolder
+            {
+            public:
+                /**
+                 * Constructor.
+                 *
+                 * @param handle Job handle.
+                 */
+                ComputeTaskHolder(int64_t handle) :
+                    handle(handle)
+                {
+                    // No-op.
+                }
+
+                /**
+                 * Destructor.
+                 */
+                virtual ~ComputeTaskHolder()
+                {
+                    // No-op.
+                }
+
+                /**
+                 * Process local job result.
+                 *
+                 * @param job Job.
+                 * @return Policy.
+                 */
+                virtual int32_t JobResultLocal(ComputeJobHolder& job) = 0;
+
+                /**
+                 * Process remote job result.
+                 *
+                 * @param job Job.
+                 * @param reader Reader for stream with result.
+                 * @return Policy.
+                 */
+                virtual int32_t JobResultRemote(ComputeJobHolder& job, binary::BinaryReaderImpl& reader) = 0;
+
+                /**
+                 * Reduce results of related jobs.
+                 */
+                virtual void Reduce() = 0;
+
+                /**
+                 * Get related job handle.
+                 *
+                 * @return Job handle.
+                 */
+                int64_t GetJobHandle()
+                {
+                    return handle;
+                }
+
+            private:
+                /** Related job handle. */
+                int64_t handle;
+            };
+
+            /**
+             * Compute task holder type-specific implementation.
+             */
+            template<typename F, typename R>
+            class ComputeTaskHolderImpl : public ComputeTaskHolder
+            {
+            public:
+                typedef F JobType;
+                typedef R ResultType;
+
+                /**
+                 * Constructor.
+                 *
+                 * @param handle Job handle.
+                 */
+                ComputeTaskHolderImpl(int64_t handle) :
+                    ComputeTaskHolder(handle)
+                {
+                    // No-op.
+                }
+
+                /**
+                 * Destructor.
+                 */
+                virtual ~ComputeTaskHolderImpl()
+                {
+                    // No-op.
+                }
+
+                /**
+                 * Process local job result.
+                 *
+                 * @param job Job.
+                 * @return Policy.
+                 */
+                virtual int32_t JobResultLocal(ComputeJobHolder& job)
+                {
+                    typedef ComputeJobHolderImpl<JobType, ResultType> ActualComputeJobHolder;
+
+                    ActualComputeJobHolder& job0 = static_cast<ActualComputeJobHolder&>(job);
+
+                    res = job0.GetResult();
+
+                    return ComputeJobResultPolicy::WAIT;
+                }
+
+                /**
+                 * Process remote job result.
+                 *
+                 * @param job Job.
+                 * @param reader Reader for stream with result.
+                 * @return Policy.
+                 */
+                virtual int32_t JobResultRemote(ComputeJobHolder& job, binary::BinaryReaderImpl& reader)
+                {
+                    res.Read(reader);
+
+                    return ComputeJobResultPolicy::WAIT;
+                }
+
+                /**
+                 * Reduce results of related jobs.
+                 */
+                virtual void Reduce()
+                {
+                    res.SetPromise(promise);
+                }
+
+                /**
+                 * Get result promise.
+                 *
+                 * @return Reference to result promise.
+                 */
+                common::Promise<ResultType>& GetPromise()
+                {
+                    return promise;
+                }
+
+            private:
+                /** Result. */
+                ComputeJobResult<ResultType> res;
+
+                /** Task result promise. */
+                common::Promise<ResultType> promise;
+            };
+        }
+    }
+}
+
+#endif //_IGNITE_IMPL_COMPUTE_COMPUTE_TASK_IMPL

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9c96de5/modules/platforms/cpp/core/include/ignite/impl/ignite_binding_impl.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/impl/ignite_binding_impl.h b/modules/platforms/cpp/core/include/ignite/impl/ignite_binding_impl.h
index a99855a..d0de432 100644
--- a/modules/platforms/cpp/core/include/ignite/impl/ignite_binding_impl.h
+++ b/modules/platforms/cpp/core/include/ignite/impl/ignite_binding_impl.h
@@ -52,6 +52,8 @@ namespace ignite
                     CACHE_ENTRY_FILTER_CREATE = 2,
 
                     CACHE_ENTRY_FILTER_APPLY = 3,
+
+                    COMPUTE_JOB_CREATE = 4,
                 };
             };
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9c96de5/modules/platforms/cpp/core/include/ignite/impl/ignite_environment.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/impl/ignite_environment.h b/modules/platforms/cpp/core/include/ignite/impl/ignite_environment.h
index e3cb859..13f7b80 100644
--- a/modules/platforms/cpp/core/include/ignite/impl/ignite_environment.h
+++ b/modules/platforms/cpp/core/include/ignite/impl/ignite_environment.h
@@ -209,6 +209,75 @@ namespace ignite
              */
             common::concurrent::SharedPointer<IgniteBindingImpl> GetBinding() const;
 
+            /**
+             * Get processor compute.
+             *
+             * @param proj Projection.
+             * @return Processor compute.
+             */
+            jobject GetProcessorCompute(jobject proj);
+
+            /**
+             * Locally execute compute job.
+             *
+             * @param jobHandle Job handle.
+             */
+            void ComputeJobExecuteLocal(int64_t jobHandle);
+
+            /**
+             * Locally commit job execution result for the task.
+             *
+             * @param taskHandle Task handle.
+             * @param jobHandle Job handle.
+             * @return Reduce politics.
+             */
+            int32_t ComputeTaskLocalJobResult(int64_t taskHandle, int64_t jobHandle);
+
+            /**
+             * Reduce compute task.
+             *
+             * @param taskHandle Task handle.
+             */
+            void ComputeTaskReduce(int64_t taskHandle);
+
+            /**
+             * Complete compute task.
+             *
+             * @param taskHandle Task handle.
+             */
+            void ComputeTaskComplete(int64_t taskHandle);
+
+            /**
+             * Create compute job.
+             *
+             * @param mem Memory.
+             * @return Job handle.
+             */
+            int64_t ComputeJobCreate(common::concurrent::SharedPointer<interop::InteropMemory>& mem);
+
+            /**
+             * Execute compute job.
+             *
+             * @param mem Memory.
+             * @return Job handle.
+             */
+            void ComputeJobExecute(common::concurrent::SharedPointer<interop::InteropMemory>& mem);
+
+            /**
+             * Destroy compute job.
+             *
+             * @param jobHandle Job handle to destroy.
+             */
+            void ComputeJobDestroy(int64_t jobHandle);
+
+            /**
+             * Consume result of remote job execution.
+             *
+             * @param mem Memory containing result.
+             * @return Reduce policy.
+             */
+            int32_t ComputeTaskJobResult(common::concurrent::SharedPointer<interop::InteropMemory>& mem);
+
         private:
             /** Node configuration. */
             IgniteConfiguration* cfg;

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9c96de5/modules/platforms/cpp/core/include/ignite/impl/ignite_impl.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/impl/ignite_impl.h b/modules/platforms/cpp/core/include/ignite/impl/ignite_impl.h
index 5b1f527..baddec4 100644
--- a/modules/platforms/cpp/core/include/ignite/impl/ignite_impl.h
+++ b/modules/platforms/cpp/core/include/ignite/impl/ignite_impl.h
@@ -22,10 +22,11 @@
 #include <ignite/jni/java.h>
 #include <ignite/common/utils.h>
 
+#include <ignite/impl/ignite_environment.h>
 #include <ignite/impl/cache/cache_impl.h>
 #include <ignite/impl/transactions/transactions_impl.h>
 #include <ignite/impl/cluster/cluster_group_impl.h>
-#include <ignite/impl/ignite_environment.h>
+#include <ignite/impl/compute/compute_impl.h>
 
 namespace ignite 
 {
@@ -38,7 +39,8 @@ namespace ignite
         {
             typedef common::concurrent::SharedPointer<IgniteEnvironment> SP_IgniteEnvironment;
             typedef common::concurrent::SharedPointer<transactions::TransactionsImpl> SP_TransactionsImpl;
-            typedef common::concurrent::SharedPointer<cluster::ClusterGroupImpl> SP_ClusterGroupImpl;
+            typedef common::concurrent::SharedPointer<compute::ComputeImpl> SP_ComputeImpl;
+            typedef common::concurrent::SharedPointer<IgniteBindingImpl> SP_IgniteBindingImpl;
         public:
             /**
              * Constructor used to create new instance.
@@ -154,7 +156,7 @@ namespace ignite
              *
              * @return IgniteBinding class instance.
              */
-            common::concurrent::SharedPointer<IgniteBindingImpl> GetBinding();
+            SP_IgniteBindingImpl GetBinding();
 
             /**
              * Get instance of the implementation from the proxy class.
@@ -185,7 +187,7 @@ namespace ignite
              *
              * @return TransactionsImpl instance.
              */
-            SP_TransactionsImpl GetTransactions() const
+            SP_TransactionsImpl GetTransactions()
             {
                 return txImpl;
             }
@@ -195,11 +197,18 @@ namespace ignite
              *
              * @return ClusterGroupImpl instance.
              */
-            SP_ClusterGroupImpl GetProjection() const
+            cluster::SP_ClusterGroupImpl GetProjection()
             {
                 return prjImpl;
             }
 
+            /**
+             * Get compute.
+             *
+             * @return ComputeImpl instance.
+             */
+            SP_ComputeImpl GetCompute();
+
         private:
             /**
              * Get transactions internal call.
@@ -213,7 +222,7 @@ namespace ignite
              *
              * @return ClusterGroupImpl instance.
              */
-            SP_ClusterGroupImpl InternalGetProjection(IgniteError &err);
+            cluster::SP_ClusterGroupImpl InternalGetProjection(IgniteError &err);
 
             /** Environment. */
             SP_IgniteEnvironment env;
@@ -225,7 +234,7 @@ namespace ignite
             SP_TransactionsImpl txImpl;
 
             /** Projection implementation. */
-            SP_ClusterGroupImpl prjImpl;
+            cluster::SP_ClusterGroupImpl prjImpl;
 
             IGNITE_NO_COPY_ASSIGNMENT(IgniteImpl)
         };

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9c96de5/modules/platforms/cpp/core/include/ignite/impl/interop/interop_target.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/impl/interop/interop_target.h b/modules/platforms/cpp/core/include/ignite/impl/interop/interop_target.h
index f9b2b7f..0384dcc 100644
--- a/modules/platforms/cpp/core/include/ignite/impl/interop/interop_target.h
+++ b/modules/platforms/cpp/core/include/ignite/impl/interop/interop_target.h
@@ -62,7 +62,7 @@ namespace ignite
                 /**
                  * Destructor.
                  */
-                ~InteropTarget();
+                virtual ~InteropTarget();
 
                 /**
                  * Internal out operation.
@@ -135,6 +135,15 @@ namespace ignite
                 OperationResult::Type InStreamOutLong(int32_t opType, InteropMemory& outInMem, IgniteError& err);
 
                 /**
+                 * In stream out object operation.
+                 *
+                 * @param opType Type of operation.
+                 * @param outInMem Input and output memory.
+                 * @return Java object references.
+                 */
+                jobject InStreamOutObject(int32_t opType, InteropMemory& outInMem);
+
+                /**
                 * Internal out-in operation.
                 *
                 * @param opType Operation type.

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9c96de5/modules/platforms/cpp/core/project/vs/core.vcxproj
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/project/vs/core.vcxproj b/modules/platforms/cpp/core/project/vs/core.vcxproj
index 5cd49f3..9911ffe 100644
--- a/modules/platforms/cpp/core/project/vs/core.vcxproj
+++ b/modules/platforms/cpp/core/project/vs/core.vcxproj
@@ -208,6 +208,8 @@
     <ClInclude Include="..\..\include\ignite\cache\query\query_sql.h" />
     <ClInclude Include="..\..\include\ignite\cache\query\query_sql_fields.h" />
     <ClInclude Include="..\..\include\ignite\cache\query\query_text.h" />
+    <ClInclude Include="..\..\include\ignite\compute\compute.h" />
+    <ClInclude Include="..\..\include\ignite\compute\compute_func.h" />
     <ClInclude Include="..\..\include\ignite\ignite_binding_context.h" />
     <ClInclude Include="..\..\include\ignite\ignite.h" />
     <ClInclude Include="..\..\include\ignite\ignite_configuration.h" />
@@ -225,6 +227,11 @@
     <ClInclude Include="..\..\include\ignite\impl\cache\query\query_fields_row_impl.h" />
     <ClInclude Include="..\..\include\ignite\impl\cache\query\query_impl.h" />
     <ClInclude Include="..\..\include\ignite\impl\cluster\cluster_group_impl.h" />
+    <ClInclude Include="..\..\include\ignite\impl\compute\cancelable_impl.h" />
+    <ClInclude Include="..\..\include\ignite\impl\compute\compute_impl.h" />
+    <ClInclude Include="..\..\include\ignite\impl\compute\compute_job_holder.h" />
+    <ClInclude Include="..\..\include\ignite\impl\compute\compute_job_result.h" />
+    <ClInclude Include="..\..\include\ignite\impl\compute\compute_task_holder.h" />
     <ClInclude Include="..\..\include\ignite\impl\helpers.h" />
     <ClInclude Include="..\..\include\ignite\impl\ignite_environment.h" />
     <ClInclude Include="..\..\include\ignite\impl\ignite_impl.h" />
@@ -251,6 +258,8 @@
     <ClCompile Include="..\..\src\impl\cache\query\continuous\continuous_query_handle_impl.cpp" />
     <ClCompile Include="..\..\src\impl\cache\query\query_impl.cpp" />
     <ClCompile Include="..\..\src\impl\cluster\cluster_group_impl.cpp" />
+    <ClCompile Include="..\..\src\impl\compute\cancelable_impl.cpp" />
+    <ClCompile Include="..\..\src\impl\compute\compute_impl.cpp" />
     <ClCompile Include="..\..\src\impl\ignite_binding_impl.cpp" />
     <ClCompile Include="..\..\src\impl\ignite_environment.cpp" />
     <ClCompile Include="..\..\src\impl\ignite_impl.cpp" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9c96de5/modules/platforms/cpp/core/project/vs/core.vcxproj.filters
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/project/vs/core.vcxproj.filters b/modules/platforms/cpp/core/project/vs/core.vcxproj.filters
index 98099a9..7b84494 100644
--- a/modules/platforms/cpp/core/project/vs/core.vcxproj.filters
+++ b/modules/platforms/cpp/core/project/vs/core.vcxproj.filters
@@ -55,6 +55,12 @@
     <ClCompile Include="..\..\src\impl\ignite_binding_impl.cpp">
       <Filter>Code\impl</Filter>
     </ClCompile>
+    <ClCompile Include="..\..\src\impl\compute\compute_impl.cpp">
+      <Filter>Code\impl\compute</Filter>
+    </ClCompile>
+    <ClCompile Include="..\..\src\impl\compute\cancelable_impl.cpp">
+      <Filter>Code\impl\compute</Filter>
+    </ClCompile>
   </ItemGroup>
   <ItemGroup>
     <ClInclude Include="..\..\include\ignite\impl\cache\cache_impl.h">
@@ -210,6 +216,27 @@
     <ClInclude Include="..\..\include\ignite\impl\helpers.h">
       <Filter>Code\impl</Filter>
     </ClInclude>
+    <ClInclude Include="..\..\include\ignite\compute\compute.h">
+      <Filter>Code\compute</Filter>
+    </ClInclude>
+    <ClInclude Include="..\..\include\ignite\impl\compute\compute_impl.h">
+      <Filter>Code\impl\compute</Filter>
+    </ClInclude>
+    <ClInclude Include="..\..\include\ignite\compute\compute_func.h">
+      <Filter>Code\compute</Filter>
+    </ClInclude>
+    <ClInclude Include="..\..\include\ignite\impl\compute\cancelable_impl.h">
+      <Filter>Code\impl\compute</Filter>
+    </ClInclude>
+    <ClInclude Include="..\..\include\ignite\impl\compute\compute_job_holder.h">
+      <Filter>Code\impl\compute</Filter>
+    </ClInclude>
+    <ClInclude Include="..\..\include\ignite\impl\compute\compute_job_result.h">
+      <Filter>Code\impl\compute</Filter>
+    </ClInclude>
+    <ClInclude Include="..\..\include\ignite\impl\compute\compute_task_holder.h">
+      <Filter>Code\impl\compute</Filter>
+    </ClInclude>
   </ItemGroup>
   <ItemGroup>
     <Filter Include="Code">
@@ -257,5 +284,11 @@
     <Filter Include="Code\impl\cache\event">
       <UniqueIdentifier>{9c5e9732-755a-4553-8926-b4cf3b6abaf3}</UniqueIdentifier>
     </Filter>
+    <Filter Include="Code\compute">
+      <UniqueIdentifier>{f1b7ced1-0e6e-4e07-a5b6-04b076797c6f}</UniqueIdentifier>
+    </Filter>
+    <Filter Include="Code\impl\compute">
+      <UniqueIdentifier>{ef20cfe1-cd30-429d-a241-575696df8399}</UniqueIdentifier>
+    </Filter>
   </ItemGroup>
 </Project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9c96de5/modules/platforms/cpp/core/src/ignite.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/src/ignite.cpp b/modules/platforms/cpp/core/src/ignite.cpp
index 2665916..9c42f1d 100644
--- a/modules/platforms/cpp/core/src/ignite.cpp
+++ b/modules/platforms/cpp/core/src/ignite.cpp
@@ -55,6 +55,11 @@ namespace ignite
         return transactions::Transactions(txImpl);
     }
 
+    compute::Compute Ignite::GetCompute()
+    {
+        return compute::Compute(impl.Get()->GetCompute());
+    }
+
     IgniteBinding Ignite::GetBinding()
     {
         return impl.Get()->GetBinding();

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9c96de5/modules/platforms/cpp/core/src/impl/cluster/cluster_group_impl.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/src/impl/cluster/cluster_group_impl.cpp b/modules/platforms/cpp/core/src/impl/cluster/cluster_group_impl.cpp
index 1bddeac..c34e828 100644
--- a/modules/platforms/cpp/core/src/impl/cluster/cluster_group_impl.cpp
+++ b/modules/platforms/cpp/core/src/impl/cluster/cluster_group_impl.cpp
@@ -37,7 +37,7 @@ namespace ignite
             ClusterGroupImpl::ClusterGroupImpl(SP_IgniteEnvironment env, jobject javaRef) :
                 InteropTarget(env, javaRef)
             {
-                // No-op.
+                computeImpl = InternalGetCompute();
             }
 
             ClusterGroupImpl::~ClusterGroupImpl()
@@ -45,22 +45,33 @@ namespace ignite
                 // No-op.
             }
 
-            ClusterGroupImpl::SP_ClusterGroupImpl ClusterGroupImpl::ForServers(IgniteError& err)
+            SP_ClusterGroupImpl ClusterGroupImpl::ForServers()
             {
-                JniErrorInfo jniErr;
+                IgniteError err;
 
                 jobject res = InOpObject(Command::FOR_SERVERS, err);
 
-                if (jniErr.code != java::IGNITE_JNI_ERR_SUCCESS)
-                    return SP_ClusterGroupImpl();
+                IgniteError::ThrowIfNeeded(err);
 
                 return FromTarget(res);
             }
 
-            ClusterGroupImpl::SP_ClusterGroupImpl ClusterGroupImpl::FromTarget(jobject javaRef)
+            ClusterGroupImpl::SP_ComputeImpl ClusterGroupImpl::GetCompute()
+            {
+                return computeImpl;
+            }
+
+            SP_ClusterGroupImpl ClusterGroupImpl::FromTarget(jobject javaRef)
             {
                 return SP_ClusterGroupImpl(new ClusterGroupImpl(GetEnvironmentPointer(), javaRef));
             }
+
+            ClusterGroupImpl::SP_ComputeImpl ClusterGroupImpl::InternalGetCompute()
+            {
+                jobject computeProc = GetEnvironment().GetProcessorCompute(GetTarget());
+
+                return SP_ComputeImpl(new compute::ComputeImpl(GetEnvironmentPointer(), computeProc));
+            }
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9c96de5/modules/platforms/cpp/core/src/impl/compute/cancelable_impl.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/src/impl/compute/cancelable_impl.cpp b/modules/platforms/cpp/core/src/impl/compute/cancelable_impl.cpp
new file mode 100644
index 0000000..6e61cc8
--- /dev/null
+++ b/modules/platforms/cpp/core/src/impl/compute/cancelable_impl.cpp
@@ -0,0 +1,59 @@
+/*
+ * 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.
+ */
+
+#include <ignite/impl/compute/cancelable_impl.h>
+
+using namespace ignite::common::concurrent;
+
+namespace
+{
+    /**
+     * Operation type.
+     */
+    struct Operation
+    {
+        enum Type
+        {
+            Cancel = 1
+        };
+    };
+}
+
+namespace ignite
+{
+    namespace impl
+    {
+        namespace compute
+        {
+            CancelableImpl::CancelableImpl(SharedPointer<IgniteEnvironment> env, jobject javaRef) :
+                InteropTarget(env, javaRef),
+                Cancelable()
+            {
+                // No-op.
+            }
+
+            void CancelableImpl::Cancel()
+            {
+                IgniteError err;
+
+                OutInOpLong(Operation::Cancel, 0, err);
+
+                IgniteError::ThrowIfNeeded(err);
+            }
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9c96de5/modules/platforms/cpp/core/src/impl/compute/compute_impl.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/src/impl/compute/compute_impl.cpp b/modules/platforms/cpp/core/src/impl/compute/compute_impl.cpp
new file mode 100644
index 0000000..591dd1f
--- /dev/null
+++ b/modules/platforms/cpp/core/src/impl/compute/compute_impl.cpp
@@ -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.
+ */
+
+#include <ignite/impl/compute/compute_impl.h>
+
+using namespace ignite::common::concurrent;
+
+namespace ignite
+{
+    namespace impl
+    {
+        namespace compute
+        {
+            ComputeImpl::ComputeImpl(SharedPointer<IgniteEnvironment> env, jobject javaRef) :
+                InteropTarget(env, javaRef)
+            {
+                // No-op.
+            }
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9c96de5/modules/platforms/cpp/core/src/impl/ignite_environment.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/src/impl/ignite_environment.cpp b/modules/platforms/cpp/core/src/impl/ignite_environment.cpp
index 2231003..4e78f09 100644
--- a/modules/platforms/cpp/core/src/impl/ignite_environment.cpp
+++ b/modules/platforms/cpp/core/src/impl/ignite_environment.cpp
@@ -20,10 +20,10 @@
 #include <ignite/impl/binary/binary_type_updater_impl.h>
 #include <ignite/impl/module_manager.h>
 #include <ignite/impl/ignite_binding_impl.h>
+#include <ignite/impl/compute/compute_task_holder.h>
 
 #include <ignite/binary/binary.h>
 #include <ignite/cache/query/continuous/continuous_query.h>
-#include <ignite/ignite_binding.h>
 #include <ignite/ignite_binding_context.h>
 
 #include <ignite/impl/ignite_environment.h>
@@ -47,13 +47,21 @@ namespace ignite
             enum Type
             {
                 CACHE_INVOKE = 8,
+                COMPUTE_TASK_JOB_RESULT = 10,
+                COMPUTE_TASK_REDUCE = 11,
+                COMPUTE_TASK_COMPLETE = 12,
+                COMPUTE_JOB_CREATE = 14,
+                COMPUTE_JOB_EXECUTE = 15,
+                COMPUTE_JOB_DESTROY = 17,
                 CONTINUOUS_QUERY_LISTENER_APPLY = 18,
                 CONTINUOUS_QUERY_FILTER_CREATE = 19,
                 CONTINUOUS_QUERY_FILTER_APPLY = 20,
                 CONTINUOUS_QUERY_FILTER_RELEASE = 21,
                 REALLOC = 36,
                 ON_START = 49,
-                ON_STOP = 50 
+                ON_STOP = 50,
+                COMPUTE_TASK_LOCAL_JOB_RESULT = 60,
+                COMPUTE_JOB_EXECUTE_LOCAL = 61
             };
         };
 
@@ -78,6 +86,47 @@ namespace ignite
                     break;
                 }
 
+                case OperationCallback::COMPUTE_JOB_CREATE:
+                {
+                    SharedPointer<InteropMemory> mem = env->Get()->GetMemory(val);
+
+                    res = env->Get()->ComputeJobCreate(mem);
+
+                    break;
+                }
+
+                case OperationCallback::COMPUTE_JOB_EXECUTE:
+                {
+                    SharedPointer<InteropMemory> mem = env->Get()->GetMemory(val);
+
+                    env->Get()->ComputeJobExecute(mem);
+
+                    break;
+                }
+
+                case OperationCallback::COMPUTE_JOB_DESTROY:
+                {
+                    env->Get()->ComputeJobDestroy(val);
+
+                    break;
+                }
+
+                case OperationCallback::COMPUTE_TASK_JOB_RESULT:
+                {
+                    SharedPointer<InteropMemory> mem = env->Get()->GetMemory(val);
+
+                    res = env->Get()->ComputeTaskJobResult(mem);
+
+                    break;
+                }
+
+                case OperationCallback::COMPUTE_TASK_REDUCE:
+                {
+                    env->Get()->ComputeTaskReduce(val);
+
+                    break;
+                }
+
                 case OperationCallback::CONTINUOUS_QUERY_LISTENER_APPLY:
                 {
                     SharedPointer<InteropMemory> mem = env->Get()->GetMemory(val);
@@ -142,10 +191,32 @@ namespace ignite
         long long IGNITE_CALL InLongLongLongObjectOutLong(void* target, int type, long long val1, long long val2, 
             long long val3, void* arg)
         {
+            int64_t res = 0;
             SharedPointer<IgniteEnvironment>* env = static_cast<SharedPointer<IgniteEnvironment>*>(target);
 
             switch (type)
             {
+                case OperationCallback::COMPUTE_JOB_EXECUTE_LOCAL:
+                {
+                    env->Get()->ComputeJobExecuteLocal(val1);
+
+                    break;
+                }
+
+                case OperationCallback::COMPUTE_TASK_LOCAL_JOB_RESULT:
+                {
+                    res = env->Get()->ComputeTaskLocalJobResult(val1, val2);
+
+                    break;
+                }
+
+                case OperationCallback::COMPUTE_TASK_COMPLETE:
+                {
+                    env->Get()->ComputeTaskComplete(val1);
+
+                    break;
+                }
+
                 case OperationCallback::ON_START:
                 {
                     env->Get()->OnStartCallback(val1, reinterpret_cast<jobject>(arg));
@@ -168,7 +239,7 @@ namespace ignite
                 }
             }
 
-            return 0;
+            return res;
         }
 
         IgniteEnvironment::IgniteEnvironment(const IgniteConfiguration& cfg) :
@@ -300,6 +371,189 @@ namespace ignite
             return binding;
         }
 
+        jobject IgniteEnvironment::GetProcessorCompute(jobject proj)
+        {
+            JniErrorInfo jniErr;
+
+            jobject res = ctx.Get()->ProcessorCompute(proc.Get(), proj, &jniErr);
+
+            IgniteError err;
+
+            IgniteError::SetError(jniErr.code, jniErr.errCls, jniErr.errMsg, err);
+
+            IgniteError::ThrowIfNeeded(err);
+
+            return res;
+        }
+
+        void IgniteEnvironment::ComputeJobExecuteLocal(int64_t jobHandle)
+        {
+            SharedPointer<compute::ComputeJobHolder> job0 =
+                StaticPointerCast<compute::ComputeJobHolder>(registry.Get(jobHandle));
+
+            compute::ComputeJobHolder* job = job0.Get();
+
+            if (job)
+                job->ExecuteLocal();
+            else
+            {
+                IGNITE_ERROR_FORMATTED_1(IgniteError::IGNITE_ERR_COMPUTE_USER_UNDECLARED_EXCEPTION,
+                    "Job is not registred for handle", "jobHandle", jobHandle);
+            }
+        }
+
+        int32_t IgniteEnvironment::ComputeTaskLocalJobResult(int64_t taskHandle, int64_t jobHandle)
+        {
+            SharedPointer<compute::ComputeJobHolder> job0 =
+                StaticPointerCast<compute::ComputeJobHolder>(registry.Get(jobHandle));
+
+            compute::ComputeJobHolder* job = job0.Get();
+
+            SharedPointer<compute::ComputeTaskHolder> task0 =
+                StaticPointerCast<compute::ComputeTaskHolder>(registry.Get(taskHandle));
+
+            compute::ComputeTaskHolder* task = task0.Get();
+
+            if (task && job)
+                return task->JobResultLocal(*job);
+
+            if (!task)
+            {
+                IGNITE_ERROR_FORMATTED_1(IgniteError::IGNITE_ERR_COMPUTE_USER_UNDECLARED_EXCEPTION,
+                    "Task is not registred for handle", "taskHandle", taskHandle);
+            }
+
+            IGNITE_ERROR_FORMATTED_1(IgniteError::IGNITE_ERR_COMPUTE_USER_UNDECLARED_EXCEPTION,
+                "Job is not registred for handle", "jobHandle", jobHandle);
+        }
+
+        void IgniteEnvironment::ComputeTaskReduce(int64_t taskHandle)
+        {
+            SharedPointer<compute::ComputeTaskHolder> task0 =
+                StaticPointerCast<compute::ComputeTaskHolder>(registry.Get(taskHandle));
+
+            compute::ComputeTaskHolder* task = task0.Get();
+
+            if (task)
+                task->Reduce();
+            else
+            {
+                IGNITE_ERROR_FORMATTED_1(IgniteError::IGNITE_ERR_COMPUTE_USER_UNDECLARED_EXCEPTION,
+                    "Task is not registred for handle", "taskHandle", taskHandle);
+            }
+        }
+
+        void IgniteEnvironment::ComputeTaskComplete(int64_t taskHandle)
+        {
+            SharedPointer<compute::ComputeTaskHolder> task0 =
+                StaticPointerCast<compute::ComputeTaskHolder>(registry.Get(taskHandle));
+
+            compute::ComputeTaskHolder* task = task0.Get();
+
+            if (task)
+            {
+                registry.Release(task->GetJobHandle());
+                registry.Release(taskHandle);
+            }
+        }
+
+        int64_t IgniteEnvironment::ComputeJobCreate(SharedPointer<InteropMemory>& mem)
+        {
+            if (!binding.Get())
+                throw IgniteError(IgniteError::IGNITE_ERR_UNKNOWN, "IgniteBinding is not initialized.");
+
+            InteropInputStream inStream(mem.Get());
+            BinaryReaderImpl reader(&inStream);
+
+            InteropOutputStream outStream(mem.Get());
+            BinaryWriterImpl writer(&outStream, GetTypeManager());
+
+            BinaryObjectImpl binJob = BinaryObjectImpl::FromMemory(*mem.Get(), inStream.Position(), 0);
+
+            int32_t jobTypeId = binJob.GetTypeId();
+
+            bool invoked = false;
+
+            int64_t handle = binding.Get()->InvokeCallback(invoked,
+                IgniteBindingImpl::CallbackType::COMPUTE_JOB_CREATE, jobTypeId, reader, writer);
+
+            if (!invoked)
+            {
+                IGNITE_ERROR_FORMATTED_1(IgniteError::IGNITE_ERR_COMPUTE_USER_UNDECLARED_EXCEPTION,
+                    "C++ compute job is not registered on the node (did you compile your program without -rdynamic?).",
+                    "jobTypeId", jobTypeId);
+            }
+
+            return handle;
+        }
+
+        void IgniteEnvironment::ComputeJobExecute(SharedPointer<InteropMemory>& mem)
+        {
+            InteropInputStream inStream(mem.Get());
+
+            InteropOutputStream outStream(mem.Get());
+            BinaryWriterImpl writer(&outStream, GetTypeManager());
+
+            int64_t jobHandle = inStream.ReadInt64();
+
+            SharedPointer<compute::ComputeJobHolder> job0 =
+                StaticPointerCast<compute::ComputeJobHolder>(registry.Get(jobHandle));
+
+            compute::ComputeJobHolder* job = job0.Get();
+
+            if (job)
+                job->ExecuteRemote(writer);
+            else
+            {
+                IGNITE_ERROR_FORMATTED_1(IgniteError::IGNITE_ERR_COMPUTE_USER_UNDECLARED_EXCEPTION,
+                    "Job is not registred for handle", "jobHandle", jobHandle);
+            }
+
+            outStream.Synchronize();
+        }
+
+        void IgniteEnvironment::ComputeJobDestroy(int64_t jobHandle)
+        {
+            registry.Release(jobHandle);
+        }
+
+        int32_t IgniteEnvironment::ComputeTaskJobResult(SharedPointer<InteropMemory>& mem)
+        {
+            InteropInputStream inStream(mem.Get());
+            BinaryReaderImpl reader(&inStream);
+
+            int64_t taskHandle = reader.ReadInt64();
+            int64_t jobHandle = reader.ReadInt64();
+
+            // Node GUID
+            reader.ReadGuid();
+
+            // Cancel flag
+            reader.ReadBool();
+
+            SharedPointer<compute::ComputeJobHolder> job0 =
+                StaticPointerCast<compute::ComputeJobHolder>(registry.Get(jobHandle));
+
+            compute::ComputeJobHolder* job = job0.Get();
+
+            SharedPointer<compute::ComputeTaskHolder> task0 =
+                StaticPointerCast<compute::ComputeTaskHolder>(registry.Get(taskHandle));
+
+            compute::ComputeTaskHolder* task = task0.Get();
+
+            if (task && job)
+                return task->JobResultRemote(*job, reader);
+
+            if (!task)
+            {
+                IGNITE_ERROR_FORMATTED_1(IgniteError::IGNITE_ERR_COMPUTE_USER_UNDECLARED_EXCEPTION,
+                    "Task is not registred for handle", "taskHandle", taskHandle);
+            }
+
+            IGNITE_ERROR_FORMATTED_1(IgniteError::IGNITE_ERR_COMPUTE_USER_UNDECLARED_EXCEPTION,
+                "Job is not registred for handle", "jobHandle", jobHandle);
+        }
+
         void IgniteEnvironment::ProcessorReleaseStart()
         {
             if (proc.Get())

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9c96de5/modules/platforms/cpp/core/src/impl/ignite_impl.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/src/impl/ignite_impl.cpp b/modules/platforms/cpp/core/src/impl/ignite_impl.cpp
index 546cd01..16e954c 100644
--- a/modules/platforms/cpp/core/src/impl/ignite_impl.cpp
+++ b/modules/platforms/cpp/core/src/impl/ignite_impl.cpp
@@ -59,11 +59,18 @@ namespace ignite
             return env.Get()->Context();
         }
 
-        SharedPointer<IgniteBindingImpl> IgniteImpl::GetBinding()
+        IgniteImpl::SP_IgniteBindingImpl IgniteImpl::GetBinding()
         {
             return env.Get()->GetBinding();
         }
 
+        IgniteImpl::SP_ComputeImpl IgniteImpl::GetCompute()
+        {
+            cluster::SP_ClusterGroupImpl serversCluster = prjImpl.Get()->ForServers();
+
+            return serversCluster.Get()->GetCompute();
+        }
+
         IgniteImpl::SP_TransactionsImpl IgniteImpl::InternalGetTransactions(IgniteError &err)
         {
             SP_TransactionsImpl res;
@@ -80,16 +87,16 @@ namespace ignite
             return res;
         }
 
-        IgniteImpl::SP_ClusterGroupImpl IgniteImpl::InternalGetProjection(IgniteError& err)
+        cluster::SP_ClusterGroupImpl IgniteImpl::InternalGetProjection(IgniteError& err)
         {
-            SP_ClusterGroupImpl res;
+            cluster::SP_ClusterGroupImpl res;
 
             JniErrorInfo jniErr;
 
             jobject txJavaRef = env.Get()->Context()->ProcessorProjection(javaRef, &jniErr);
 
             if (txJavaRef)
-                res = SP_ClusterGroupImpl(new cluster::ClusterGroupImpl(env, txJavaRef));
+                res = cluster::SP_ClusterGroupImpl(new cluster::ClusterGroupImpl(env, txJavaRef));
             else
                 IgniteError::SetError(jniErr.code, jniErr.errCls, jniErr.errMsg, err);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9c96de5/modules/platforms/cpp/core/src/impl/interop/interop_target.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/src/impl/interop/interop_target.cpp b/modules/platforms/cpp/core/src/impl/interop/interop_target.cpp
index b0932e7..7eed6f3 100644
--- a/modules/platforms/cpp/core/src/impl/interop/interop_target.cpp
+++ b/modules/platforms/cpp/core/src/impl/interop/interop_target.cpp
@@ -216,6 +216,26 @@ namespace ignite
                 return OperationResult::AI_ERROR;
             }
 
+            jobject InteropTarget::InStreamOutObject(int32_t opType, InteropMemory& outInMem)
+            {
+                JniErrorInfo jniErr;
+
+                int64_t outInPtr = outInMem.PointerLong();
+
+                if (outInPtr)
+                {
+                    jobject res = env.Get()->Context()->TargetInStreamOutObject(javaRef, opType, outInPtr, &jniErr);
+
+                    IgniteError err;
+                    IgniteError::SetError(jniErr.code, jniErr.errCls, jniErr.errMsg, err);
+                    IgniteError::ThrowIfNeeded(err);
+
+                    return res;
+                }
+
+                return 0;
+            }
+
             int64_t InteropTarget::OutInOpLong(int32_t opType, int64_t val, IgniteError& err)
             {
                 JniErrorInfo jniErr;

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9c96de5/modules/platforms/cpp/core/src/impl/transactions/transactions_impl.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/src/impl/transactions/transactions_impl.cpp b/modules/platforms/cpp/core/src/impl/transactions/transactions_impl.cpp
index 133b375..b9e976a 100644
--- a/modules/platforms/cpp/core/src/impl/transactions/transactions_impl.cpp
+++ b/modules/platforms/cpp/core/src/impl/transactions/transactions_impl.cpp
@@ -165,8 +165,6 @@ namespace ignite
 
             TransactionState::Type TransactionsImpl::TxCommit(int64_t id, IgniteError& err)
             {
-                JniErrorInfo jniErr;
-
                 int state = static_cast<int>(OutInOpLong(Operation::COMMIT, id, err));
 
                 return ToTransactionState(state);
@@ -174,8 +172,6 @@ namespace ignite
 
             TransactionState::Type TransactionsImpl::TxRollback(int64_t id, IgniteError& err)
             {
-                JniErrorInfo jniErr;
-
                 int state = static_cast<int>(OutInOpLong(Operation::ROLLBACK, id, err));
 
                 return ToTransactionState(state);
@@ -183,8 +179,6 @@ namespace ignite
 
             TransactionState::Type TransactionsImpl::TxClose(int64_t id, IgniteError& err)
             {
-                JniErrorInfo jniErr;
-
                 int state = static_cast<int>(OutInOpLong(Operation::CLOSE, id, err));
 
                 return ToTransactionState(state);
@@ -192,8 +186,6 @@ namespace ignite
 
             bool TransactionsImpl::TxSetRollbackOnly(int64_t id, IgniteError& err)
             {
-                JniErrorInfo jniErr;
-
                 bool rollbackOnly = OutInOpLong(Operation::SET_ROLLBACK_ONLY, id, err) == 1;
 
                 return rollbackOnly;
@@ -201,8 +193,6 @@ namespace ignite
 
             TransactionState::Type TransactionsImpl::TxState(int64_t id, IgniteError& err)
             {
-                JniErrorInfo jniErr;
-
                 int state = static_cast<int>(OutInOpLong(Operation::STATE, id, err));
 
                 return ToTransactionState(state);

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9c96de5/modules/platforms/cpp/jni/include/ignite/jni/java.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/jni/include/ignite/jni/java.h b/modules/platforms/cpp/jni/include/ignite/jni/java.h
index 85955b3..f6d7207 100644
--- a/modules/platforms/cpp/jni/include/ignite/jni/java.h
+++ b/modules/platforms/cpp/jni/include/ignite/jni/java.h
@@ -365,6 +365,7 @@ namespace ignite
                 jobject ProcessorDataStreamer(jobject obj, const char* name, bool keepPortable);
                 jobject ProcessorTransactions(jobject obj, JniErrorInfo* errInfo = NULL);
                 jobject ProcessorCompute(jobject obj, jobject prj);
+                jobject ProcessorCompute(jobject obj, jobject prj, JniErrorInfo* errInfo);
                 jobject ProcessorMessage(jobject obj, jobject prj);
                 jobject ProcessorEvents(jobject obj, jobject prj);
                 jobject ProcessorServices(jobject obj, jobject prj);

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9c96de5/modules/platforms/cpp/jni/src/java.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/jni/src/java.cpp b/modules/platforms/cpp/jni/src/java.cpp
index 809aa17..bc6af34 100644
--- a/modules/platforms/cpp/jni/src/java.cpp
+++ b/modules/platforms/cpp/jni/src/java.cpp
@@ -1154,6 +1154,16 @@ namespace ignite
                 return LocalToGlobal(env, res);
             }
 
+            jobject JniContext::ProcessorCompute(jobject obj, jobject prj, JniErrorInfo* errInfo) {
+                JNIEnv* env = Attach();
+
+                jobject res = env->CallObjectMethod(obj, jvm->GetMembers().m_PlatformProcessor_compute, prj);
+
+                ExceptionCheck(env, errInfo);
+
+                return LocalToGlobal(env, res);
+            }
+
             jobject JniContext::ProcessorMessage(jobject obj, jobject prj) {
                 JNIEnv* env = Attach();
 


[10/19] ignite git commit: ignite-5203 Simple BLOB support added

Posted by sb...@apache.org.
ignite-5203 Simple BLOB support added


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

Branch: refs/heads/ignite-5075
Commit: 64ab5cdfe9297ed819d269166a3d1a3b489d03ec
Parents: 618a448
Author: agura <ag...@apache.org>
Authored: Thu May 18 19:40:09 2017 +0300
Committer: agura <ag...@apache.org>
Committed: Tue May 30 00:33:46 2017 +0300

----------------------------------------------------------------------
 .../jdbc2/JdbcAbstractDmlStatementSelfTest.java |  50 +-
 .../JdbcAbstractUpdateStatementSelfTest.java    |  11 +-
 .../ignite/internal/jdbc2/JdbcBlobTest.java     | 485 +++++++++++++++++++
 .../jdbc2/JdbcInsertStatementSelfTest.java      |  16 +-
 .../jdbc2/JdbcMergeStatementSelfTest.java       |  16 +-
 .../jdbc2/JdbcPreparedStatementSelfTest.java    |  47 ++
 .../jdbc/suite/IgniteJdbcDriverTestSuite.java   |   7 +-
 .../apache/ignite/internal/jdbc2/JdbcBlob.java  | 191 ++++++++
 .../ignite/internal/jdbc2/JdbcConnection.java   |   2 +-
 .../internal/jdbc2/JdbcPreparedStatement.java   |   4 +-
 .../ignite/internal/jdbc2/JdbcResultSet.java    |   8 +-
 11 files changed, 808 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/64ab5cdf/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractDmlStatementSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractDmlStatementSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractDmlStatementSelfTest.java
index 6ef86d3..81c913d 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractDmlStatementSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractDmlStatementSelfTest.java
@@ -18,8 +18,11 @@
 package org.apache.ignite.internal.jdbc2;
 
 import java.io.Serializable;
+import java.io.UnsupportedEncodingException;
+import java.sql.Blob;
 import java.sql.Connection;
 import java.sql.DriverManager;
+import java.sql.SQLException;
 import java.util.Collections;
 import org.apache.ignite.cache.QueryEntity;
 import org.apache.ignite.cache.query.annotations.QuerySqlField;
@@ -35,6 +38,9 @@ import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
  * Statement test.
  */
 public abstract class JdbcAbstractDmlStatementSelfTest extends GridCommonAbstractTest {
+    /** UTF 16 character set name. */
+    private static final String UTF_16 = "UTF-16"; // RAWTOHEX function use UTF-16 for conversion strings to byte arrays.
+
     /** JDBC URL. */
     private static final String BASE_URL = CFG_URL_PREFIX + "cache=" + DEFAULT_CACHE_NAME + "@modules/clients/src/test/config/jdbc-config.xml";
 
@@ -42,7 +48,7 @@ public abstract class JdbcAbstractDmlStatementSelfTest extends GridCommonAbstrac
     static final String BASE_URL_BIN = CFG_URL_PREFIX + "cache=" + DEFAULT_CACHE_NAME + "@modules/clients/src/test/config/jdbc-bin-config.xml";
 
     /** SQL SELECT query for verification. */
-    static final String SQL_SELECT = "select _key, id, firstName, lastName, age from Person";
+    static final String SQL_SELECT = "select _key, id, firstName, lastName, age, data from Person";
 
     /** Alias for _key */
     private static final String KEY_ALIAS = "key";
@@ -107,6 +113,7 @@ public abstract class JdbcAbstractDmlStatementSelfTest extends GridCommonAbstrac
         e.addQueryField("age", Integer.class.getName(), null);
         e.addQueryField("firstName", String.class.getName(), null);
         e.addQueryField("lastName", String.class.getName(), null);
+        e.addQueryField("data", byte[].class.getName(), null);
 
         cache.setQueryEntities(Collections.singletonList(e));
 
@@ -136,6 +143,42 @@ public abstract class JdbcAbstractDmlStatementSelfTest extends GridCommonAbstrac
     }
 
     /**
+     * @param str String.
+     */
+    static byte[] getBytes(String str) {
+        try {
+            return str.getBytes(UTF_16);
+        }
+        catch (UnsupportedEncodingException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    /**
+     * @param blob Blob.
+     */
+    static byte[] getBytes(Blob blob) {
+        try {
+            return blob.getBytes(1, (int)blob.length());
+        }
+        catch (SQLException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    /**
+     * @param arr Array.
+     */
+    static String str(byte[] arr) {
+        try {
+            return new String(arr, UTF_16);
+        }
+        catch (UnsupportedEncodingException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    /**
      * Person.
      */
     @SuppressWarnings("UnusedDeclaration")
@@ -156,6 +199,10 @@ public abstract class JdbcAbstractDmlStatementSelfTest extends GridCommonAbstrac
         @QuerySqlField
         private final int age;
 
+        /** Binary data. */
+        @QuerySqlField
+        private final byte[] data;
+
         /**
          * @param id ID.
          * @param firstName First name.
@@ -171,6 +218,7 @@ public abstract class JdbcAbstractDmlStatementSelfTest extends GridCommonAbstrac
             this.firstName = firstName;
             this.lastName = lastName;
             this.age = age;
+            this.data = getBytes(lastName);
         }
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/64ab5cdf/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractUpdateStatementSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractUpdateStatementSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractUpdateStatementSelfTest.java
index a20b815..ace1be6 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractUpdateStatementSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractUpdateStatementSelfTest.java
@@ -19,12 +19,15 @@ package org.apache.ignite.internal.jdbc2;
 
 import java.sql.Statement;
 
+/**
+ *
+ */
 public abstract class JdbcAbstractUpdateStatementSelfTest extends JdbcAbstractDmlStatementSelfTest {
     /** SQL query to populate cache. */
-    private static final String ITEMS_SQL = "insert into Person(_key, id, firstName, lastName, age) values " +
-        "('p1', 1, 'John', 'White', 25), " +
-        "('p2', 2, 'Joe', 'Black', 35), " +
-        "('p3', 3, 'Mike', 'Green', 40)";
+    private static final String ITEMS_SQL = "insert into Person(_key, id, firstName, lastName, age, data) values " +
+        "('p1', 1, 'John', 'White', 25, RAWTOHEX('White')), " +
+        "('p2', 2, 'Joe', 'Black', 35, RAWTOHEX('Black')), " +
+        "('p3', 3, 'Mike', 'Green', 40, RAWTOHEX('Green'))";
 
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {

http://git-wip-us.apache.org/repos/asf/ignite/blob/64ab5cdf/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcBlobTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcBlobTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcBlobTest.java
new file mode 100644
index 0000000..9e0e0d2
--- /dev/null
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcBlobTest.java
@@ -0,0 +1,485 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.jdbc2;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.sql.SQLException;
+import java.util.Arrays;
+import junit.framework.TestCase;
+
+/**
+ *
+ */
+public class JdbcBlobTest extends TestCase {
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLength() throws Exception {
+        JdbcBlob blob = new JdbcBlob(new byte[16]);
+
+        assertEquals(16, (int)blob.length());
+
+        blob.free();
+
+        try {
+            blob.length();
+
+            fail();
+        }
+        catch (SQLException e) {
+            // No-op.
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testGetBytes() throws Exception {
+        byte[] arr = new byte[] {0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15};
+
+        JdbcBlob blob = new JdbcBlob(arr);
+
+        try {
+            blob.getBytes(0, 16);
+
+            fail();
+        }
+        catch (SQLException e) {
+            // No-op.
+        }
+
+        try {
+            blob.getBytes(17, 16);
+
+            fail();
+        }
+        catch (SQLException e) {
+            // No-op.
+        }
+
+        try {
+            blob.getBytes(1, -1);
+
+            fail();
+        }
+        catch (SQLException e) {
+            // No-op.
+        }
+
+        byte[] res = blob.getBytes(1, 0);
+        assertEquals(0, res.length);
+
+        assertTrue(Arrays.equals(arr, blob.getBytes(1, 16)));
+
+        res = blob.getBytes(1, 20);
+        assertEquals(16, res.length);
+        assertTrue(Arrays.equals(arr, res));
+
+        res = blob.getBytes(1, 10);
+        assertEquals(10, res.length);
+        assertEquals(0, res[0]);
+        assertEquals(9, res[9]);
+
+        res = blob.getBytes(7, 10);
+        assertEquals(10, res.length);
+        assertEquals(6, res[0]);
+        assertEquals(15, res[9]);
+
+        res = blob.getBytes(7, 20);
+        assertEquals(10, res.length);
+        assertEquals(6, res[0]);
+        assertEquals(15, res[9]);
+
+        res = blob.getBytes(1, 0);
+        assertEquals(0, res.length);
+
+        blob.free();
+
+        try {
+            blob.getBytes(1, 16);
+
+            fail();
+        }
+        catch (SQLException e) {
+            // No-op.
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testGetBinaryStream() throws Exception {
+        byte[] arr = new byte[] {0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15};
+
+        JdbcBlob blob = new JdbcBlob(arr);
+
+        InputStream is = blob.getBinaryStream();
+
+        byte[] res = readBytes(is);
+
+        assertTrue(Arrays.equals(arr, res));
+
+        blob.free();
+
+        try {
+            blob.getBinaryStream();
+
+            fail();
+        }
+        catch (SQLException e) {
+            // No-op.
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testGetBinaryStreamWithParams() throws Exception {
+        byte[] arr = new byte[] {0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15};
+
+        JdbcBlob blob = new JdbcBlob(arr);
+
+        try {
+            blob.getBinaryStream(0, arr.length);
+
+            fail();
+        }
+        catch (SQLException e) {
+            // No-op.
+        }
+
+        try {
+            blob.getBinaryStream(1, 0);
+
+            fail();
+        }
+        catch (SQLException e) {
+            // No-op.
+        }
+
+        try {
+            blob.getBinaryStream(17, arr.length);
+
+            fail();
+        }
+        catch (SQLException e) {
+            // No-op.
+        }
+
+        try {
+            blob.getBinaryStream(1, arr.length + 1);
+
+            fail();
+        }
+        catch (SQLException e) {
+            // No-op.
+        }
+
+        InputStream is = blob.getBinaryStream(1, arr.length);
+        byte[] res = readBytes(is);
+        assertTrue(Arrays.equals(arr, res));
+
+        is = blob.getBinaryStream(1, 10);
+        res = readBytes(is);
+        assertEquals(10, res.length);
+        assertEquals(0, res[0]);
+        assertEquals(9, res[9]);
+
+        is = blob.getBinaryStream(6, 10);
+        res = readBytes(is);
+        assertEquals(10, res.length);
+        assertEquals(5, res[0]);
+        assertEquals(14, res[9]);
+
+        blob.free();
+
+        try {
+            blob.getBinaryStream(1, arr.length);
+
+            fail();
+        }
+        catch (SQLException e) {
+            // No-op.
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPositionBytePattern() throws Exception {
+        byte[] arr = new byte[] {0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15};
+
+        JdbcBlob blob = new JdbcBlob(arr);
+
+        assertEquals(-1, blob.position(new byte[] {1, 2, 3}, 0));
+        assertEquals(-1, blob.position(new byte[] {1, 2, 3}, arr.length + 1));
+        assertEquals(-1, blob.position(new byte[0], 1));
+        assertEquals(-1, blob.position(new byte[17], 1));
+        assertEquals(-1, blob.position(new byte[] {3, 2, 1}, 1));
+        assertEquals(1, blob.position(new byte[] {0, 1, 2}, 1));
+        assertEquals(2, blob.position(new byte[] {1, 2, 3}, 1));
+        assertEquals(2, blob.position(new byte[] {1, 2, 3}, 2));
+        assertEquals(-1, blob.position(new byte[] {1, 2, 3}, 3));
+        assertEquals(14, blob.position(new byte[] {13, 14, 15}, 3));
+        assertEquals(-1, blob.position(new byte[] {0, 1, 3}, 1));
+        assertEquals(-1, blob.position(new byte[] {0, 2, 3}, 1));
+        assertEquals(-1, blob.position(new byte[] {1, 2, 4}, 1));
+
+        blob.free();
+
+        try {
+            blob.position(new byte[] {0, 1, 2}, 1);
+
+            fail();
+        }
+        catch (SQLException e) {
+            // No-op.
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPositionBlobPattern() throws Exception {
+        byte[] arr = new byte[] {0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15};
+
+        JdbcBlob blob = new JdbcBlob(arr);
+
+        assertEquals(-1, blob.position(new JdbcBlob(new byte[] {1, 2, 3}), 0));
+        assertEquals(-1, blob.position(new JdbcBlob(new byte[] {1, 2, 3}), arr.length + 1));
+        assertEquals(-1, blob.position(new JdbcBlob(new byte[0]), 1));
+        assertEquals(-1, blob.position(new JdbcBlob(new byte[17]), 1));
+        assertEquals(-1, blob.position(new JdbcBlob(new byte[] {3, 2, 1}), 1));
+        assertEquals(1, blob.position(new JdbcBlob(new byte[] {0, 1, 2}), 1));
+        assertEquals(2, blob.position(new JdbcBlob(new byte[] {1, 2, 3}), 1));
+        assertEquals(2, blob.position(new JdbcBlob(new byte[] {1, 2, 3}), 2));
+        assertEquals(-1, blob.position(new JdbcBlob(new byte[] {1, 2, 3}), 3));
+        assertEquals(14, blob.position(new JdbcBlob(new byte[] {13, 14, 15}), 3));
+        assertEquals(-1, blob.position(new JdbcBlob(new byte[] {0, 1, 3}), 1));
+        assertEquals(-1, blob.position(new JdbcBlob(new byte[] {0, 2, 3}), 1));
+        assertEquals(-1, blob.position(new JdbcBlob(new byte[] {1, 2, 4}), 1));
+
+        blob.free();
+
+        try {
+            blob.position(new JdbcBlob(new byte[] {0, 1, 2}), 1);
+
+            fail();
+        }
+        catch (SQLException e) {
+            // No-op.
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSetBytes() throws Exception {
+        byte[] arr = new byte[] {0, 1, 2, 3, 4, 5, 6, 7};
+
+        JdbcBlob blob = new JdbcBlob(arr);
+
+        try {
+            blob.setBytes(0, new byte[4]);
+
+            fail();
+        }
+        catch (SQLException e) {
+            // No-op.
+        }
+
+        try {
+            blob.setBytes(17, new byte[4]);
+
+            fail();
+        }
+        catch (ArrayIndexOutOfBoundsException e) {
+            // No-op.
+        }
+
+        assertEquals(4, blob.setBytes(1, new byte[] {3, 2, 1, 0}));
+        assertTrue(Arrays.equals(new byte[] {3, 2, 1, 0, 4, 5, 6, 7}, blob.getBytes(1, arr.length)));
+
+        assertEquals(4, blob.setBytes(5, new byte[] {7, 6, 5, 4}));
+        assertTrue(Arrays.equals(new byte[] {3, 2, 1, 0, 7, 6, 5, 4}, blob.getBytes(1, arr.length)));
+
+        assertEquals(4, blob.setBytes(7, new byte[] {8, 9, 10, 11}));
+        assertTrue(Arrays.equals(new byte[] {3, 2, 1, 0, 7, 6, 8, 9, 10, 11}, blob.getBytes(1, (int)blob.length())));
+
+        blob = new JdbcBlob(new byte[] {15, 16});
+        assertEquals(8, blob.setBytes(1, new byte[] {0, 1, 2, 3, 4, 5, 6, 7}));
+        assertTrue(Arrays.equals(new byte[] {0, 1, 2, 3, 4, 5, 6, 7}, blob.getBytes(1, (int)blob.length())));
+
+        blob.free();
+
+        try {
+            blob.setBytes(1, new byte[] {0, 1, 2});
+
+            fail();
+        }
+        catch (SQLException e) {
+            // No-op.
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSetBytesWithOffsetAndLength() throws Exception {
+        byte[] arr = new byte[] {0, 1, 2, 3, 4, 5, 6, 7};
+
+        JdbcBlob blob = new JdbcBlob(arr);
+
+        try {
+            blob.setBytes(0, new byte[4], 0, 2);
+
+            fail();
+        }
+        catch (SQLException e) {
+            // No-op.
+        }
+
+        try {
+            blob.setBytes(17, new byte[4], 0, 2);
+
+            fail();
+        }
+        catch (ArrayIndexOutOfBoundsException e) {
+            // No-op.
+        }
+
+        try {
+            blob.setBytes(1, new byte[4], -1, 2);
+
+            fail();
+        }
+        catch (ArrayIndexOutOfBoundsException e) {
+            // No-op.
+        }
+
+        try {
+            blob.setBytes(1, new byte[4], 0, 5);
+
+            fail();
+        }
+        catch (ArrayIndexOutOfBoundsException e) {
+            // No-op.
+        }
+
+        assertEquals(4, blob.setBytes(1, new byte[] {3, 2, 1, 0}, 0, 4));
+        assertTrue(Arrays.equals(new byte[] {3, 2, 1, 0, 4, 5, 6, 7}, blob.getBytes(1, arr.length)));
+
+        assertEquals(4, blob.setBytes(5, new byte[] {7, 6, 5, 4}, 0, 4));
+        assertTrue(Arrays.equals(new byte[] {3, 2, 1, 0, 7, 6, 5, 4}, blob.getBytes(1, arr.length)));
+
+        assertEquals(4, blob.setBytes(7, new byte[] {8, 9, 10, 11}, 0, 4));
+        assertTrue(Arrays.equals(new byte[] {3, 2, 1, 0, 7, 6, 8, 9, 10, 11}, blob.getBytes(1, (int)blob.length())));
+
+        assertEquals(2, blob.setBytes(1, new byte[] {3, 2, 1, 0}, 2, 2));
+        assertTrue(Arrays.equals(new byte[] {1, 0, 1, 0, 7, 6, 8, 9, 10, 11}, blob.getBytes(1, (int)blob.length())));
+
+        assertEquals(2, blob.setBytes(9, new byte[] {3, 2, 1, 0}, 1, 2));
+        assertTrue(Arrays.equals(new byte[] {1, 0, 1, 0, 7, 6, 8, 9, 2, 1}, blob.getBytes(1, (int)blob.length())));
+
+        assertEquals(3, blob.setBytes(9, new byte[] {3, 2, 1, 0}, 0, 3));
+        assertTrue(Arrays.equals(new byte[] {1, 0, 1, 0, 7, 6, 8, 9, 3, 2, 1}, blob.getBytes(1, (int)blob.length())));
+
+        blob = new JdbcBlob(new byte[] {15, 16});
+        assertEquals(8, blob.setBytes(1, new byte[] {0, 1, 2, 3, 4, 5, 6, 7}, 0, 8));
+        assertTrue(Arrays.equals(new byte[] {0, 1, 2, 3, 4, 5, 6, 7}, blob.getBytes(1, (int)blob.length())));
+
+        blob.free();
+
+        try {
+            blob.setBytes(1, new byte[] {0, 1, 2}, 0, 2);
+
+            fail();
+        }
+        catch (SQLException e) {
+            // No-op.
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTruncate() throws Exception {
+        byte[] arr = new byte[] {0, 1, 2, 3, 4, 5, 6, 7};
+
+        JdbcBlob blob = new JdbcBlob(arr);
+
+        try {
+            blob.truncate(-1);
+
+            fail();
+        }
+        catch(SQLException e) {
+            // No-op.
+        }
+
+        try {
+            blob.truncate(arr.length + 1);
+
+            fail();
+        }
+        catch(SQLException e) {
+            // No-op.
+        }
+
+        blob.truncate(4);
+        assertTrue(Arrays.equals(new byte[] {0, 1, 2, 3}, blob.getBytes(1, (int)blob.length())));
+
+        blob.truncate(0);
+        assertEquals(0, (int)blob.length());
+
+        blob.free();
+
+        try {
+            blob.truncate(0);
+
+            fail();
+        }
+        catch (SQLException e) {
+            // No-op.
+            System.out.println();
+        }
+    }
+
+    /**
+     * @param is Input stream.
+     */
+    private static byte[] readBytes(InputStream is) throws IOException {
+        byte[] tmp = new byte[16];
+
+        int i = 0;
+        int read;
+        int cnt = 0;
+
+        while ((read = is.read()) != -1) {
+            tmp[i++] = (byte)read;
+            cnt++;
+        }
+
+        byte[] res = new byte[cnt];
+
+        System.arraycopy(tmp, 0, res, 0, cnt);
+
+        return res;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/64ab5cdf/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcInsertStatementSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcInsertStatementSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcInsertStatementSelfTest.java
index b23f947..0e7539f 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcInsertStatementSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcInsertStatementSelfTest.java
@@ -33,14 +33,14 @@ import org.apache.ignite.testframework.GridTestUtils;
  */
 public class JdbcInsertStatementSelfTest extends JdbcAbstractDmlStatementSelfTest {
     /** SQL query. */
-    private static final String SQL = "insert into Person(_key, id, firstName, lastName, age) values " +
-        "('p1', 1, 'John', 'White', 25), " +
-        "('p2', 2, 'Joe', 'Black', 35), " +
-        "('p3', 3, 'Mike', 'Green', 40)";
+    private static final String SQL = "insert into Person(_key, id, firstName, lastName, age, data) values " +
+        "('p1', 1, 'John', 'White', 25, RAWTOHEX('White')), " +
+        "('p2', 2, 'Joe', 'Black', 35, RAWTOHEX('Black')), " +
+        "('p3', 3, 'Mike', 'Green', 40, RAWTOHEX('Green'))";
 
     /** SQL query. */
-    private static final String SQL_PREPARED = "insert into Person(_key, id, firstName, lastName, age) values " +
-        "(?, ?, ?, ?, ?), (?, ?, ?, ?, ?)";
+    private static final String SQL_PREPARED = "insert into Person(_key, id, firstName, lastName, age, data) values " +
+        "(?, ?, ?, ?, ?, ?), (?, ?, ?, ?, ?, ?)";
 
     /** Statement. */
     private Statement stmt;
@@ -79,6 +79,7 @@ public class JdbcInsertStatementSelfTest extends JdbcAbstractDmlStatementSelfTes
                         assertEquals("John", rs.getString("firstName"));
                         assertEquals("White", rs.getString("lastName"));
                         assertEquals(25, rs.getInt("age"));
+                        assertEquals("White", str(getBytes(rs.getBlob("data"))));
                         break;
 
                     case 2:
@@ -86,6 +87,7 @@ public class JdbcInsertStatementSelfTest extends JdbcAbstractDmlStatementSelfTes
                         assertEquals("Joe", rs.getString("firstName"));
                         assertEquals("Black", rs.getString("lastName"));
                         assertEquals(35, rs.getInt("age"));
+                        assertEquals("Black", str(getBytes(rs.getBlob("data"))));
                         break;
 
                     case 3:
@@ -93,6 +95,7 @@ public class JdbcInsertStatementSelfTest extends JdbcAbstractDmlStatementSelfTes
                         assertEquals("Mike", rs.getString("firstName"));
                         assertEquals("Green", rs.getString("lastName"));
                         assertEquals(40, rs.getInt("age"));
+                        assertEquals("Green", str(getBytes(rs.getBlob("data"))));
                         break;
 
                     case 4:
@@ -100,6 +103,7 @@ public class JdbcInsertStatementSelfTest extends JdbcAbstractDmlStatementSelfTes
                         assertEquals("Leah", rs.getString("firstName"));
                         assertEquals("Grey", rs.getString("lastName"));
                         assertEquals(22, rs.getInt("age"));
+                        assertEquals("Grey", str(getBytes(rs.getBlob("data"))));
                         break;
 
                     default:

http://git-wip-us.apache.org/repos/asf/ignite/blob/64ab5cdf/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcMergeStatementSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcMergeStatementSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcMergeStatementSelfTest.java
index f4577f5..1432a78 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcMergeStatementSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcMergeStatementSelfTest.java
@@ -28,14 +28,14 @@ import org.apache.ignite.cache.CachePeekMode;
  */
 public class JdbcMergeStatementSelfTest extends JdbcAbstractDmlStatementSelfTest {
     /** SQL query. */
-    private static final String SQL = "merge into Person(_key, id, firstName, lastName, age) values " +
-        "('p1', 1, 'John', 'White', 25), " +
-        "('p2', 2, 'Joe', 'Black', 35), " +
-        "('p3', 3, 'Mike', 'Green', 40)";
+    private static final String SQL = "merge into Person(_key, id, firstName, lastName, age, data) values " +
+        "('p1', 1, 'John', 'White', 25, RAWTOHEX('White')), " +
+        "('p2', 2, 'Joe', 'Black', 35, RAWTOHEX('Black')), " +
+        "('p3', 3, 'Mike', 'Green', 40, RAWTOHEX('Green'))";
 
     /** SQL query. */
-    protected static final String SQL_PREPARED = "merge into Person(_key, id, firstName, lastName, age) values " +
-        "(?, ?, ?, ?, ?), (?, ?, ?, ?, ?)";
+    protected static final String SQL_PREPARED = "merge into Person(_key, id, firstName, lastName, age, data) values " +
+        "(?, ?, ?, ?, ?, ?), (?, ?, ?, ?, ?, ?)";
 
     /** Statement. */
     protected Statement stmt;
@@ -74,6 +74,7 @@ public class JdbcMergeStatementSelfTest extends JdbcAbstractDmlStatementSelfTest
                         assertEquals("John", rs.getString("firstName"));
                         assertEquals("White", rs.getString("lastName"));
                         assertEquals(25, rs.getInt("age"));
+                        assertEquals("White", str(getBytes(rs.getBlob("data"))));
                         break;
 
                     case 2:
@@ -81,6 +82,7 @@ public class JdbcMergeStatementSelfTest extends JdbcAbstractDmlStatementSelfTest
                         assertEquals("Joe", rs.getString("firstName"));
                         assertEquals("Black", rs.getString("lastName"));
                         assertEquals(35, rs.getInt("age"));
+                        assertEquals("Black", str(getBytes(rs.getBlob("data"))));
                         break;
 
                     case 3:
@@ -88,6 +90,7 @@ public class JdbcMergeStatementSelfTest extends JdbcAbstractDmlStatementSelfTest
                         assertEquals("Mike", rs.getString("firstName"));
                         assertEquals("Green", rs.getString("lastName"));
                         assertEquals(40, rs.getInt("age"));
+                        assertEquals("Green", str(getBytes(rs.getBlob("data"))));
                         break;
 
                     case 4:
@@ -95,6 +98,7 @@ public class JdbcMergeStatementSelfTest extends JdbcAbstractDmlStatementSelfTest
                         assertEquals("Leah", rs.getString("firstName"));
                         assertEquals("Grey", rs.getString("lastName"));
                         assertEquals(22, rs.getInt("age"));
+                        assertEquals("Grey", str(getBytes(rs.getBlob("data"))));
                         break;
 
                     default:

http://git-wip-us.apache.org/repos/asf/ignite/blob/64ab5cdf/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcPreparedStatementSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcPreparedStatementSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcPreparedStatementSelfTest.java
index 5e402ff..30bd018 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcPreparedStatementSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcPreparedStatementSelfTest.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.jdbc2;
 import java.io.Serializable;
 import java.math.BigDecimal;
 import java.net.URL;
+import java.sql.Blob;
 import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.PreparedStatement;
@@ -117,6 +118,7 @@ public class JdbcPreparedStatementSelfTest extends GridCommonAbstractTest {
         o.bigVal = new BigDecimal(1);
         o.strVal = "str";
         o.arrVal = new byte[] {1};
+        o.blobVal = new byte[] {1};
         o.dateVal = new Date(1);
         o.timeVal = new Time(1);
         o.tsVal = new Timestamp(1);
@@ -529,6 +531,47 @@ public class JdbcPreparedStatementSelfTest extends GridCommonAbstractTest {
     /**
      * @throws Exception If failed.
      */
+    public void testBlob() throws Exception {
+        stmt = conn.prepareStatement("select * from TestObject where blobVal is not distinct from ?");
+
+        Blob blob = conn.createBlob();
+
+        blob.setBytes(1, new byte[] {1});
+
+        stmt.setBlob(1, blob);
+
+        ResultSet rs = stmt.executeQuery();
+
+        int cnt = 0;
+
+        while (rs.next()) {
+            if (cnt == 0)
+                assert rs.getInt("id") == 1;
+
+            cnt++;
+        }
+
+        assertEquals(1, cnt);
+
+        stmt.setNull(1, BINARY);
+
+        rs = stmt.executeQuery();
+
+        cnt = 0;
+
+        while (rs.next()) {
+            if (cnt == 0)
+                assert rs.getInt("id") == 2;
+
+            cnt++;
+        }
+
+        assert cnt == 1;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testDate() throws Exception {
         stmt = conn.prepareStatement("select * from TestObject where dateVal is not distinct from ?");
 
@@ -725,6 +768,10 @@ public class JdbcPreparedStatementSelfTest extends GridCommonAbstractTest {
 
         /** */
         @QuerySqlField
+        private byte[] blobVal;
+
+        /** */
+        @QuerySqlField
         private Date dateVal;
 
         /** */

http://git-wip-us.apache.org/repos/asf/ignite/blob/64ab5cdf/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java
index e2f09ba..abfdf90 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java
@@ -18,6 +18,8 @@
 package org.apache.ignite.jdbc.suite;
 
 import junit.framework.TestSuite;
+import org.apache.ignite.internal.jdbc2.JdbcBlobTest;
+import org.apache.ignite.internal.jdbc2.JdbcDistributedJoinsQueryTest;
 import org.apache.ignite.jdbc.JdbcComplexQuerySelfTest;
 import org.apache.ignite.jdbc.JdbcConnectionSelfTest;
 import org.apache.ignite.jdbc.JdbcEmptyCacheSelfTest;
@@ -41,7 +43,7 @@ public class IgniteJdbcDriverTestSuite extends TestSuite {
     public static TestSuite suite() throws Exception {
         TestSuite suite = new TestSuite("Ignite JDBC Driver Test Suite");
 
-        // Thin client based driver tests
+        // Thin client based driver tests.
         suite.addTest(new TestSuite(JdbcConnectionSelfTest.class));
         suite.addTest(new TestSuite(JdbcStatementSelfTest.class));
         suite.addTest(new TestSuite(JdbcPreparedStatementSelfTest.class));
@@ -61,7 +63,7 @@ public class IgniteJdbcDriverTestSuite extends TestSuite {
         suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcPreparedStatementSelfTest.class));
         suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcResultSetSelfTest.class));
         suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcComplexQuerySelfTest.class));
-        suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcDistributedJoinsQueryTest.class));
+        suite.addTest(new TestSuite(JdbcDistributedJoinsQueryTest.class));
         suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcMetadataSelfTest.class));
         suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcEmptyCacheSelfTest.class));
         suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcLocalCachesSelfTest.class));
@@ -71,6 +73,7 @@ public class IgniteJdbcDriverTestSuite extends TestSuite {
         suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcInsertStatementSelfTest.class));
         suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcBinaryMarshallerInsertStatementSelfTest.class));
         suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcDeleteStatementSelfTest.class));
+        suite.addTest(new TestSuite(JdbcBlobTest.class));
         suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcStreamingSelfTest.class));
 
         // DDL tests.

http://git-wip-us.apache.org/repos/asf/ignite/blob/64ab5cdf/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcBlob.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcBlob.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcBlob.java
new file mode 100644
index 0000000..d1610d1
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcBlob.java
@@ -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.
+ */
+
+package org.apache.ignite.internal.jdbc2;
+
+import java.io.ByteArrayInputStream;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.sql.Blob;
+import java.sql.SQLException;
+import java.sql.SQLFeatureNotSupportedException;
+import java.util.Arrays;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * Simple BLOB implementation. Actually there is no such entity as BLOB in Ignite. So using arrays is preferable way
+ * to work with binary objects.
+ *
+ * This implementation can be useful for reading binary fields of objects through JDBC.
+ */
+public class JdbcBlob implements Blob {
+    /** Byte array. */
+    private byte[] arr;
+
+    /**
+     * @param arr Byte array.
+     */
+    public JdbcBlob(byte[] arr) {
+        this.arr = arr;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long length() throws SQLException {
+        ensureNotClosed();
+
+        return arr.length;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte[] getBytes(long pos, int len) throws SQLException {
+        ensureNotClosed();
+
+        if (pos < 1 || arr.length - pos < 0 || len < 0)
+            throw new SQLException("Invalid argument. Position can't be less than 1 or " +
+                "greater than size of underlying byte array. Requested length also can't be negative " + "" +
+                "[pos=" + pos + ", len=" + len +']');
+
+        int idx = (int)(pos - 1);
+
+        int size = len > arr.length - idx ? arr.length - idx : len;
+
+        byte[] res = new byte[size];
+
+        U.arrayCopy(arr, idx, res, 0, size);
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public InputStream getBinaryStream() throws SQLException {
+        ensureNotClosed();
+
+        return new ByteArrayInputStream(arr);
+    }
+
+    /** {@inheritDoc} */
+    @Override public InputStream getBinaryStream(long pos, long len) throws SQLException {
+        ensureNotClosed();
+
+        if (pos < 1 || len < 1 || pos > arr.length || len > arr.length - pos + 1)
+            throw new SQLException("Invalid argument. Position can't be less than 1 or " +
+                "greater than size of underlying byte array. Requested length can't be negative and can't be " +
+                "greater than available bytes from given position [pos=" + pos + ", len=" + len +']');
+
+
+        return new ByteArrayInputStream(arr, (int)(pos - 1), (int)len);
+    }
+
+    /** {@inheritDoc} */
+    @Override public long position(byte[] ptrn, long start) throws SQLException {
+        ensureNotClosed();
+
+        if (start < 1 || start > arr.length || ptrn.length == 0 || ptrn.length > arr.length)
+            return -1;
+
+        for(int i = 0, pos = (int)(start - 1); pos < arr.length;) {
+            if (arr[pos] == ptrn[i]) {
+                pos++;
+
+                i++;
+
+                if (i == ptrn.length)
+                    return pos - ptrn.length + 1;
+            }
+            else {
+                pos = pos - i + 1;
+
+                i = 0;
+            }
+        }
+
+        return -1;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long position(Blob ptrn, long start) throws SQLException {
+        ensureNotClosed();
+
+        if (start < 1 || start > arr.length || ptrn.length() == 0 || ptrn.length() > arr.length)
+            return -1;
+
+        return position(ptrn.getBytes(1, (int)ptrn.length()), start);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int setBytes(long pos, byte[] bytes) throws SQLException {
+        return setBytes(pos, bytes, 0, bytes.length);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int setBytes(long pos, byte[] bytes, int off, int len) throws SQLException {
+        ensureNotClosed();
+
+        if (pos < 1)
+            throw new SQLException("Invalid argument. Position can't be less than 1 [pos=" + pos + ']');
+
+        int idx = (int)(pos - 1);
+
+        if (pos - 1 > arr.length || off < 0 || off >= bytes.length || off + len > bytes.length)
+            throw new ArrayIndexOutOfBoundsException();
+
+        byte[] dst = arr;
+
+        if (idx + len > arr.length) {
+            dst = new byte[arr.length + (len - (arr.length - idx))];
+
+            U.arrayCopy(arr, 0, dst, 0, idx);
+
+            arr = dst;
+        }
+
+        U.arrayCopy(bytes, off, dst, idx, len);
+
+        return len;
+    }
+
+    /** {@inheritDoc} */
+    @Override public OutputStream setBinaryStream(long pos) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void truncate(long len) throws SQLException {
+        ensureNotClosed();
+
+        if (len < 0 || len > arr.length)
+            throw new SQLException("Invalid argument. Length can't be " +
+                "less than zero or greater than Blob length [len=" + len + ']');
+
+        arr = Arrays.copyOf(arr, (int)len);
+
+    }
+
+    /** {@inheritDoc} */
+    @Override public void free() throws SQLException {
+        if (arr != null)
+            arr = null;
+    }
+
+    /**
+     *
+     */
+    private void ensureNotClosed() throws SQLException {
+        if (arr == null)
+            throw new SQLException("Blob instance can't be used after free() has been called.");
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/64ab5cdf/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcConnection.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcConnection.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcConnection.java
index f6f79fb..ee8b605 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcConnection.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcConnection.java
@@ -597,7 +597,7 @@ public class JdbcConnection implements Connection {
     @Override public Blob createBlob() throws SQLException {
         ensureNotClosed();
 
-        throw new SQLFeatureNotSupportedException("SQL-specific types are not supported.");
+        return new JdbcBlob(new byte[0]);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/64ab5cdf/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcPreparedStatement.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcPreparedStatement.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcPreparedStatement.java
index 54e58e9..1a66ced 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcPreparedStatement.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcPreparedStatement.java
@@ -239,9 +239,7 @@ public class JdbcPreparedStatement extends JdbcStatement implements PreparedStat
 
     /** {@inheritDoc} */
     @Override public void setBlob(int paramIdx, Blob x) throws SQLException {
-        ensureNotClosed();
-
-        throw new SQLFeatureNotSupportedException("SQL-specific types are not supported.");
+        setBytes(paramIdx, x.getBytes(1, (int)x.length()));
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/64ab5cdf/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcResultSet.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcResultSet.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcResultSet.java
index 0f91bd8..10cf17a 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcResultSet.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcResultSet.java
@@ -952,9 +952,7 @@ public class JdbcResultSet implements ResultSet {
 
     /** {@inheritDoc} */
     @Override public Blob getBlob(int colIdx) throws SQLException {
-        ensureNotClosed();
-
-        throw new SQLFeatureNotSupportedException("SQL-specific types are not supported.");
+        return new JdbcBlob(getBytes(colIdx));
     }
 
     /** {@inheritDoc} */
@@ -985,9 +983,7 @@ public class JdbcResultSet implements ResultSet {
 
     /** {@inheritDoc} */
     @Override public Blob getBlob(String colLb) throws SQLException {
-        ensureNotClosed();
-
-        throw new SQLFeatureNotSupportedException("SQL-specific types are not supported.");
+        return new JdbcBlob(getBytes(colLb));
     }
 
     /** {@inheritDoc} */


[14/19] ignite git commit: IGNITE-5299 Don't set IgniteLock as broken if lock is failoverSafe

Posted by sb...@apache.org.
IGNITE-5299 Don't set IgniteLock as broken if lock is failoverSafe


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

Branch: refs/heads/ignite-5075
Commit: fd1d87b997c02e511731fcfd0c49d1ce5d0af2e6
Parents: 8237ac6
Author: Evgenii Zhuravlev <ez...@gridgain.com>
Authored: Mon May 29 16:46:11 2017 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Tue May 30 16:10:13 2017 +0300

----------------------------------------------------------------------
 .../processors/datastructures/GridCacheLockImpl.java      | 10 ++++++----
 1 file changed, 6 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/fd1d87b9/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheLockImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheLockImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheLockImpl.java
index 0192354..7bf55a3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheLockImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheLockImpl.java
@@ -1151,10 +1151,10 @@ public final class GridCacheLockImpl implements GridCacheLockEx, IgniteChangeGlo
 
         try {
             if (nodeId.equals(sync.getOwnerNode())) {
-                sync.setBroken(true);
-
-                if (!sync.failoverSafe)
+                if (!sync.failoverSafe) {
+                    sync.setBroken(true);
                     sync.interruptAll();
+                }
             }
 
             // Try to notify any waiting threads.
@@ -1173,7 +1173,9 @@ public final class GridCacheLockImpl implements GridCacheLockEx, IgniteChangeGlo
             return;
         }
 
-        sync.setBroken(true);
+        if (!sync.failoverSafe) {
+            sync.setBroken(true);
+        }
 
         sync.interruptAll();
 


[17/19] ignite git commit: IGNITE-5242: Disallowed interference between SQL and non-SQL caches lifecycle. This closes #1996.

Posted by sb...@apache.org.
IGNITE-5242: Disallowed interference between SQL and non-SQL caches lifecycle. This closes #1996.


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

Branch: refs/heads/ignite-5075
Commit: 10a0a84a15fde067b522da156d2de2bd17faedfb
Parents: a858611
Author: Alexander Paschenko <al...@gmail.com>
Authored: Wed May 31 10:19:35 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Wed May 31 10:19:35 2017 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/internal/IgniteEx.java    |  9 ++-
 .../apache/ignite/internal/IgniteKernal.java    | 19 +++--
 .../cache/CacheAffinitySharedManager.java       |  1 +
 .../internal/processors/cache/CacheData.java    | 13 ++++
 .../cache/CacheJoinNodeDiscoveryData.java       | 15 +++-
 .../processors/cache/ClusterCachesInfo.java     | 47 +++++++++---
 .../cache/DynamicCacheChangeRequest.java        | 21 +++++-
 .../cache/DynamicCacheDescriptor.java           | 13 ++++
 .../processors/cache/GridCacheAttributes.java   | 14 +++-
 .../processors/cache/GridCacheProcessor.java    | 53 ++++++++++---
 .../processors/cache/IgniteCacheProxy.java      |  2 +-
 .../datastructures/DataStructuresProcessor.java |  1 +
 .../processors/query/GridQueryProcessor.java    |  4 +-
 .../handlers/cache/GridCacheCommandHandler.java |  2 +-
 .../cache/IgniteDynamicCacheStartSelfTest.java  |  4 +-
 .../processors/igfs/IgfsIgniteMock.java         |  4 +-
 .../junits/multijvm/IgniteProcessProxy.java     |  4 +-
 .../cache/index/H2DynamicTableSelfTest.java     | 79 +++++++++++++++++++-
 18 files changed, 260 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/10a0a84a/modules/core/src/main/java/org/apache/ignite/internal/IgniteEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteEx.java
index 8c27027..775f493 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteEx.java
@@ -71,20 +71,23 @@ public interface IgniteEx extends Ignite {
      * of the existing cache.
      *
      * @param cacheCfg Cache configuration to use.
+     * @param sql {@code true} if this call is triggered by SQL command {@code CREATE TABLE}, {@code false} otherwise.
      * @return Tuple [Existing or newly created cache; {@code true} if cache was newly crated, {@code false} otherwise]
      * @throws CacheException If error occurs.
      */
-    public <K, V> IgniteBiTuple<IgniteCache<K, V>, Boolean> getOrCreateCache0(CacheConfiguration<K, V> cacheCfg)
-        throws CacheException;
+    public <K, V> IgniteBiTuple<IgniteCache<K, V>, Boolean> getOrCreateCache0(CacheConfiguration<K, V> cacheCfg,
+        boolean sql) throws CacheException;
 
     /**
      * Stops dynamically started cache.
      *
      * @param cacheName Cache name to stop.
+     * @param sql {@code true} if only cache created with SQL command {@code CREATE TABLE} should be affected,
+     *     {@code false} otherwise.
      * @return {@code true} if cache has been stopped as the result of this call, {@code false} otherwise.
      * @throws CacheException If error occurs.
      */
-    public boolean destroyCache0(String cacheName) throws CacheException;
+    public boolean destroyCache0(String cacheName, boolean sql) throws CacheException;
 
     /**
      * Checks if the event type is user-recordable.

http://git-wip-us.apache.org/repos/asf/ignite/blob/10a0a84a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index 6393e9f..6d05147 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -2791,13 +2791,13 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
 
     /** {@inheritDoc} */
     @Override public <K, V> IgniteCache<K, V> getOrCreateCache(CacheConfiguration<K, V> cacheCfg) {
-        return getOrCreateCache0(cacheCfg).get1();
+        return getOrCreateCache0(cacheCfg, false).get1();
     }
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public <K, V> IgniteBiTuple<IgniteCache<K, V>, Boolean> getOrCreateCache0(
-        CacheConfiguration<K, V> cacheCfg) {
+        CacheConfiguration<K, V> cacheCfg, boolean sql) {
         A.notNull(cacheCfg, "cacheCfg");
         CU.validateCacheName(cacheCfg.getName());
 
@@ -2809,7 +2809,9 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
             Boolean res = false;
 
             if (ctx.cache().cache(cacheCfg.getName()) == null) {
-                res = ctx.cache().dynamicStartCache(cacheCfg,
+                res =
+                    sql ? ctx.cache().dynamicStartSqlCache(cacheCfg).get() :
+                    ctx.cache().dynamicStartCache(cacheCfg,
                     cacheCfg.getName(),
                     null,
                     false,
@@ -3017,14 +3019,14 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
 
     /** {@inheritDoc} */
     @Override public void destroyCache(String cacheName) {
-        destroyCache0(cacheName);
+        destroyCache0(cacheName, false);
     }
 
     /** {@inheritDoc} */
-    @Override public boolean destroyCache0(String cacheName) throws CacheException {
+    @Override public boolean destroyCache0(String cacheName, boolean sql) throws CacheException {
         CU.validateCacheName(cacheName);
 
-        IgniteInternalFuture<Boolean> stopFut = destroyCacheAsync(cacheName, true);
+        IgniteInternalFuture<Boolean> stopFut = destroyCacheAsync(cacheName, sql, true);
 
         try {
             return stopFut.get();
@@ -3050,10 +3052,11 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
 
     /**
      * @param cacheName Cache name.
+     * @param sql If the cache needs to be destroyed only if it was created by SQL {@code CREATE TABLE} command.
      * @param checkThreadTx If {@code true} checks that current thread does not have active transactions.
      * @return Ignite future.
      */
-    public IgniteInternalFuture<Boolean> destroyCacheAsync(String cacheName, boolean checkThreadTx) {
+    public IgniteInternalFuture<Boolean> destroyCacheAsync(String cacheName, boolean sql, boolean checkThreadTx) {
         CU.validateCacheName(cacheName);
 
         guard();
@@ -3061,7 +3064,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         try {
             checkClusterState();
 
-            return ctx.cache().dynamicDestroyCache(cacheName, checkThreadTx);
+            return ctx.cache().dynamicDestroyCache(cacheName, sql, checkThreadTx);
         }
         finally {
             unguard();

http://git-wip-us.apache.org/repos/asf/ignite/blob/10a0a84a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
index cec42a8..889eeda 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
@@ -339,6 +339,7 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
                 false,
                 action.descriptor().receivedFrom(),
                 action.descriptor().staticallyConfigured(),
+                action.descriptor().sql(),
                 req.deploymentId(),
                 req.schema());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/10a0a84a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheData.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheData.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheData.java
index 4768a9a..3e2c259 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheData.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheData.java
@@ -53,6 +53,9 @@ public class CacheData implements Serializable {
     private final boolean staticCfg;
 
     /** */
+    private final boolean sql;
+
+    /** */
     private final boolean template;
 
     /** Flags added for future usage. */
@@ -66,6 +69,7 @@ public class CacheData implements Serializable {
      * @param schema Query schema.
      * @param rcvdFrom Node ID cache was started from.
      * @param staticCfg {@code True} if cache was statically configured.
+     * @param sql {@code True} if cache was created by an SQL command such as {@code CREATE TABLE}.
      * @param template {@code True} if this is cache template.
      * @param flags Flags (added for future usage).
      */
@@ -76,6 +80,7 @@ public class CacheData implements Serializable {
         QuerySchema schema,
         UUID rcvdFrom,
         boolean staticCfg,
+        boolean sql,
         boolean template,
         byte flags) {
         assert cacheCfg != null;
@@ -90,6 +95,7 @@ public class CacheData implements Serializable {
         this.schema = schema;
         this.rcvdFrom = rcvdFrom;
         this.staticCfg = staticCfg;
+        this.sql = sql;
         this.template = template;
         this.flags = flags;
     }
@@ -130,6 +136,13 @@ public class CacheData implements Serializable {
     }
 
     /**
+     * @return {@code True} if cache was created by an SQL command such as {@code CREATE TABLE}.
+     */
+    public boolean sql() {
+        return sql;
+    }
+
+    /**
      * @return Cache configuration.
      */
     public CacheConfiguration cacheConfiguration() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/10a0a84a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheJoinNodeDiscoveryData.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheJoinNodeDiscoveryData.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheJoinNodeDiscoveryData.java
index ea24140..afc01c9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheJoinNodeDiscoveryData.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheJoinNodeDiscoveryData.java
@@ -106,17 +106,23 @@ class CacheJoinNodeDiscoveryData implements Serializable {
         @GridToStringInclude
         private final CacheType cacheType;
 
+        /** */
+        @GridToStringInclude
+        private final boolean sql;
+
         /** Flags added for future usage. */
         private final byte flags;
 
         /**
          * @param ccfg Cache configuration.
          * @param cacheType Cache type.
+         * @param sql SQL flag - {@code true} if cache was created with {@code CREATE TABLE}.
          * @param flags Flags (for future usage).
          */
-        CacheInfo(CacheConfiguration ccfg, CacheType cacheType, byte flags) {
+        CacheInfo(CacheConfiguration ccfg, CacheType cacheType, boolean sql, byte flags) {
             this.ccfg = ccfg;
             this.cacheType = cacheType;
+            this.sql = sql;
             this.flags = flags;
         }
 
@@ -134,6 +140,13 @@ class CacheJoinNodeDiscoveryData implements Serializable {
             return cacheType;
         }
 
+        /**
+         * @return SQL flag - {@code true} if cache was created with {@code CREATE TABLE}.
+         */
+        boolean sql() {
+            return sql;
+        }
+
         /** {@inheritDoc} */
         @Override public String toString() {
             return S.toString(CacheInfo.class, this);

http://git-wip-us.apache.org/repos/asf/ignite/blob/10a0a84a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
index c372037..7ff5622 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
@@ -52,7 +52,7 @@ import static org.apache.ignite.events.EventType.EVT_NODE_JOINED;
 import static org.apache.ignite.internal.GridComponent.DiscoveryDataExchangeType.CACHE_PROC;
 
 /**
- * Logic related to cache discovery date processing.
+ * Logic related to cache discovery data processing.
  */
 class ClusterCachesInfo {
     /** */
@@ -112,7 +112,7 @@ class ClusterCachesInfo {
                 CacheData cacheData = gridData.caches().get(locCfg.getName());
 
                 if (cacheData != null)
-                    checkCache(locCfg, cacheData.cacheConfiguration(), cacheData.receivedFrom());
+                    checkCache(locCacheInfo, cacheData, cacheData.receivedFrom());
             }
         }
 
@@ -122,19 +122,23 @@ class ClusterCachesInfo {
     /**
      * Checks that remote caches has configuration compatible with the local.
      *
-     * @param locCfg Local configuration.
-     * @param rmtCfg Remote configuration.
+     * @param locInfo Local configuration.
+     * @param rmtData Remote configuration.
      * @param rmt Remote node.
      * @throws IgniteCheckedException If check failed.
      */
-    private void checkCache(CacheConfiguration<?, ?> locCfg, CacheConfiguration<?, ?> rmtCfg, UUID rmt)
+    @SuppressWarnings("unchecked")
+    private void checkCache(CacheJoinNodeDiscoveryData.CacheInfo locInfo, CacheData rmtData, UUID rmt)
         throws IgniteCheckedException {
-        GridCacheAttributes rmtAttr = new GridCacheAttributes(rmtCfg);
-        GridCacheAttributes locAttr = new GridCacheAttributes(locCfg);
+        GridCacheAttributes rmtAttr = new GridCacheAttributes(rmtData.cacheConfiguration(), rmtData.sql());
+        GridCacheAttributes locAttr = new GridCacheAttributes(locInfo.config(), locInfo.sql());
 
         CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "cacheMode", "Cache mode",
             locAttr.cacheMode(), rmtAttr.cacheMode(), true);
 
+        CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "sql", "SQL flag",
+            locAttr.sql(), rmtAttr.sql(), true);
+
         if (rmtAttr.cacheMode() != LOCAL) {
             CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "interceptor", "Cache Interceptor",
                 locAttr.interceptorClassName(), rmtAttr.interceptorClassName(), true);
@@ -147,8 +151,8 @@ class ClusterCachesInfo {
 
             ClusterNode rmtNode = ctx.discovery().node(rmt);
 
-            if (CU.affinityNode(ctx.discovery().localNode(), locCfg.getNodeFilter())
-                && rmtNode != null && CU.affinityNode(rmtNode, rmtCfg.getNodeFilter())) {
+            if (CU.affinityNode(ctx.discovery().localNode(), locInfo.config().getNodeFilter())
+                && rmtNode != null && CU.affinityNode(rmtNode, rmtData.cacheConfiguration().getNodeFilter())) {
                 CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "storeFactory", "Store factory",
                     locAttr.storeFactoryClassName(), rmtAttr.storeFactoryClassName(), true);
             }
@@ -248,6 +252,7 @@ class ClusterCachesInfo {
                         true,
                         req.initiatingNodeId(),
                         false,
+                        false,
                         req.deploymentId(),
                         req.schema());
 
@@ -287,6 +292,7 @@ class ClusterCachesInfo {
                             false,
                             req.initiatingNodeId(),
                             false,
+                            req.sql(),
                             req.deploymentId(),
                             req.schema());
 
@@ -377,6 +383,22 @@ class ClusterCachesInfo {
                 assert req.stop() ^ req.close() : req;
 
                 if (desc != null) {
+                    if (req.sql() && !desc.sql()) {
+                        ctx.cache().completeCacheStartFuture(req, false,
+                            new IgniteCheckedException("Only cache created with CREATE TABLE may be removed with " +
+                                "DROP TABLE [cacheName=" + req.cacheName() + ']'));
+
+                        continue;
+                    }
+
+                    if (!req.sql() && desc.sql()) {
+                        ctx.cache().completeCacheStartFuture(req, false,
+                            new IgniteCheckedException("Only cache created with cache API may be removed with " +
+                                "direct call to destroyCache [cacheName=" + req.cacheName() + ']'));
+
+                        continue;
+                    }
+
                     DynamicCacheDescriptor old = registeredCaches.remove(req.cacheName());
 
                     assert old != null : "Dynamic cache map was concurrently modified [req=" + req + ']';
@@ -587,6 +609,7 @@ class ClusterCachesInfo {
                 desc.schema(),
                 desc.receivedFrom(),
                 desc.staticallyConfigured(),
+                desc.sql(),
                 false,
                 (byte)0);
 
@@ -603,6 +626,7 @@ class ClusterCachesInfo {
                 desc.schema(),
                 desc.receivedFrom(),
                 desc.staticallyConfigured(),
+                false,
                 true,
                 (byte)0);
 
@@ -632,6 +656,7 @@ class ClusterCachesInfo {
                 true,
                 cacheData.receivedFrom(),
                 cacheData.staticallyConfigured(),
+                false,
                 cacheData.deploymentId(),
                 cacheData.schema());
 
@@ -648,6 +673,7 @@ class ClusterCachesInfo {
                 false,
                 cacheData.receivedFrom(),
                 cacheData.staticallyConfigured(),
+                cacheData.sql(),
                 cacheData.deploymentId(),
                 cacheData.schema());
 
@@ -707,6 +733,7 @@ class ClusterCachesInfo {
                             desc.template(),
                             desc.receivedFrom(),
                             desc.staticallyConfigured(),
+                            desc.sql(),
                             desc.deploymentId(),
                             desc.schema());
 
@@ -784,6 +811,7 @@ class ClusterCachesInfo {
                     true,
                     nodeId,
                     true,
+                    false,
                     joinData.cacheDeploymentId(),
                     new QuerySchema(cfg.getQueryEntities()));
 
@@ -803,6 +831,7 @@ class ClusterCachesInfo {
                     false,
                     nodeId,
                     true,
+                    cacheInfo.sql(),
                     joinData.cacheDeploymentId(),
                     new QuerySchema(cfg.getQueryEntities()));
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/10a0a84a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeRequest.java
index f8c2c7d..5434061 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeRequest.java
@@ -70,6 +70,9 @@ public class DynamicCacheChangeRequest implements Serializable {
     /** Close flag. */
     private boolean close;
 
+    /** Whether cache was created through SQL. */
+    private boolean sql;
+
     /** Fail if exists flag. */
     private boolean failIfExists;
 
@@ -165,12 +168,14 @@ public class DynamicCacheChangeRequest implements Serializable {
     /**
      * @param ctx Context.
      * @param cacheName Cache name.
+     * @param sql {@code true} if the cache must be stopped only if it was created by SQL command {@code CREATE TABLE}.
      * @param destroy Destroy flag.
      * @return Cache stop request.
      */
-    static DynamicCacheChangeRequest stopRequest(GridKernalContext ctx, String cacheName, boolean destroy) {
+    static DynamicCacheChangeRequest stopRequest(GridKernalContext ctx, String cacheName, boolean sql, boolean destroy) {
         DynamicCacheChangeRequest req = new DynamicCacheChangeRequest(UUID.randomUUID(), cacheName, ctx.localNodeId());
 
+        req.sql(sql);
         req.stop(true);
         req.destroy(destroy);
 
@@ -381,6 +386,20 @@ public class DynamicCacheChangeRequest implements Serializable {
     }
 
     /**
+     * @return SQL flag.
+     */
+    public boolean sql() {
+        return sql;
+    }
+
+    /**
+     * @param sql New SQL flag.
+     */
+    public void sql(boolean sql) {
+        this.sql = sql;
+    }
+
+    /**
      * @param nodeId ID of node provided cache configuration in discovery data.
      */
     public void receivedFrom(UUID nodeId) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/10a0a84a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheDescriptor.java
index 40d3706..a682f63 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheDescriptor.java
@@ -48,6 +48,9 @@ public class DynamicCacheDescriptor {
     /** Statically configured flag. */
     private final boolean staticCfg;
 
+    /** SQL flag - whether the cache is created by an SQL command such as {@code CREATE TABLE}. */
+    private boolean sql;
+
     /** Cache type. */
     private CacheType cacheType;
 
@@ -97,6 +100,7 @@ public class DynamicCacheDescriptor {
      * @param template {@code True} if this is template configuration.
      * @param rcvdFrom ID of node provided cache configuration
      * @param staticCfg {@code True} if cache statically configured.
+     * @param sql SQL flag - whether the cache is created by an SQL command such as {@code CREATE TABLE}.
      * @param deploymentId Deployment ID.
      * @param schema Query schema.
      */
@@ -107,6 +111,7 @@ public class DynamicCacheDescriptor {
         boolean template,
         UUID rcvdFrom,
         boolean staticCfg,
+        boolean sql,
         IgniteUuid deploymentId,
         QuerySchema schema) {
         assert cacheCfg != null;
@@ -123,6 +128,7 @@ public class DynamicCacheDescriptor {
         this.template = template;
         this.rcvdFrom = rcvdFrom;
         this.staticCfg = staticCfg;
+        this.sql = sql;
         this.deploymentId = deploymentId;
 
         pluginMgr = new CachePluginManager(ctx, cacheCfg);
@@ -170,6 +176,13 @@ public class DynamicCacheDescriptor {
     }
 
     /**
+     * @return SQL flag.
+     */
+    public boolean sql() {
+        return sql;
+    }
+
+    /**
      * @return Cache name.
      */
     public String cacheName() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/10a0a84a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
index 1caf60d..4af2518 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
@@ -47,11 +47,16 @@ public class GridCacheAttributes implements Serializable {
     /** Cache configuration. */
     private CacheConfiguration ccfg;
 
+    /** SQL flag - whether the cache is created by an SQL command such as {@code CREATE TABLE}. */
+    private boolean sql;
+
     /**
      * @param cfg Cache configuration.
      */
-    public GridCacheAttributes(CacheConfiguration cfg) {
+    public GridCacheAttributes(CacheConfiguration cfg, boolean sql) {
         ccfg = cfg;
+
+        this.sql = sql;
     }
 
     /**
@@ -273,6 +278,13 @@ public class GridCacheAttributes implements Serializable {
     }
 
     /**
+     * @return SQL flag.
+     */
+    public boolean sql() {
+        return sql;
+    }
+
+    /**
      * @param obj Object to get class of.
      * @return Class name or {@code null}.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/10a0a84a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index 9a25672..dbcddf4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -120,6 +120,7 @@ import org.apache.ignite.internal.util.typedef.CIX1;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -675,11 +676,12 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
     /**
      * @param cfg Cache configuration.
+     * @param sql SQL flag - {@code true} if cache was created with {@code CREATE TABLE}.
      * @param caches Caches map.
      * @param templates Templates map.
      * @throws IgniteCheckedException If failed.
      */
-    private void addCacheOnJoin(CacheConfiguration cfg,
+    private void addCacheOnJoin(CacheConfiguration cfg, boolean sql,
         Map<String, CacheJoinNodeDiscoveryData.CacheInfo> caches,
         Map<String, CacheJoinNodeDiscoveryData.CacheInfo> templates) throws IgniteCheckedException {
         CU.validateCacheName(cfg.getName());
@@ -716,10 +718,10 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             else
                 stopSeq.addFirst(cfg.getName());
 
-            caches.put(cfg.getName(), new CacheJoinNodeDiscoveryData.CacheInfo(cfg, cacheType, (byte)0));
+            caches.put(cfg.getName(), new CacheJoinNodeDiscoveryData.CacheInfo(cfg, cacheType, sql, (byte)0));
         }
         else
-            templates.put(cfg.getName(), new CacheJoinNodeDiscoveryData.CacheInfo(cfg, CacheType.USER, (byte)0));
+            templates.put(cfg.getName(), new CacheJoinNodeDiscoveryData.CacheInfo(cfg, CacheType.USER, false, (byte)0));
     }
 
     /**
@@ -740,7 +742,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
             cfgs[i] = cfg; // Replace original configuration value.
 
-            addCacheOnJoin(cfg, caches, templates);
+            addCacheOnJoin(cfg, false, caches, templates);
         }
     }
 
@@ -769,8 +771,9 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                 for (String name : savedCacheNames) {
                     CacheConfiguration cfg = sharedCtx.pageStore().readConfiguration(name);
 
+                    // TODO IGNITE-5306 - set correct SQL flag below.
                     if (cfg != null)
-                        addCacheOnJoin(cfg, caches, templates);
+                        addCacheOnJoin(cfg, false, caches, templates);
                 }
             }
         }
@@ -2218,18 +2221,41 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             cacheName,
             nearCfg,
             CacheType.USER,
+            false,
             failIfExists,
             failIfNotStarted,
             checkThreadTx);
     }
 
     /**
+     * Dynamically starts cache as a result of SQL {@code CREATE TABLE} command.
+     *
+     * @param ccfg Cache configuration.
+     */
+    @SuppressWarnings("IfMayBeConditional")
+    public IgniteInternalFuture<Boolean> dynamicStartSqlCache(
+        CacheConfiguration ccfg
+    ) {
+        A.notNull(ccfg, "ccfg");
+
+        return dynamicStartCache(ccfg,
+            ccfg.getName(),
+            ccfg.getNearConfiguration(),
+            CacheType.USER,
+            true,
+            false,
+            true,
+            true);
+    }
+
+    /**
      * Dynamically starts cache.
      *
      * @param ccfg Cache configuration.
      * @param cacheName Cache name.
      * @param nearCfg Near cache configuration.
      * @param cacheType Cache type.
+     * @param sql If the cache needs to be created as the result of SQL {@code CREATE TABLE} command.
      * @param failIfExists Fail if exists flag.
      * @param failIfNotStarted If {@code true} fails if cache is not started.
      * @param checkThreadTx If {@code true} checks that current thread does not have active transactions.
@@ -2241,6 +2267,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         String cacheName,
         @Nullable NearCacheConfiguration nearCfg,
         CacheType cacheType,
+        boolean sql,
         boolean failIfExists,
         boolean failIfNotStarted,
         boolean checkThreadTx
@@ -2256,6 +2283,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                 cacheName,
                 nearCfg,
                 cacheType,
+                sql,
                 failIfExists,
                 failIfNotStarted);
 
@@ -2309,6 +2337,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                     ccfg.getName(),
                     null,
                     cacheType,
+                    false,
                     failIfExists,
                     true
                 );
@@ -2337,16 +2366,18 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
     /**
      * @param cacheName Cache name to destroy.
+     * @param sql If the cache needs to be destroyed only if it was created as the result
+     *     of SQL {@code CREATE TABLE} command.
      * @param checkThreadTx If {@code true} checks that current thread does not have active transactions.
      * @return Future that will be completed when cache is destroyed.
      */
-    public IgniteInternalFuture<Boolean> dynamicDestroyCache(String cacheName, boolean checkThreadTx) {
+    public IgniteInternalFuture<Boolean> dynamicDestroyCache(String cacheName, boolean sql, boolean checkThreadTx) {
         assert cacheName != null;
 
         if (checkThreadTx)
             checkEmptyTransactions();
 
-        DynamicCacheChangeRequest req = DynamicCacheChangeRequest.stopRequest(ctx, cacheName, true);
+        DynamicCacheChangeRequest req = DynamicCacheChangeRequest.stopRequest(ctx, cacheName, sql, true);
 
         return F.first(initiateCacheChanges(F.asList(req), false));
     }
@@ -2363,7 +2394,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         List<DynamicCacheChangeRequest> reqs = new ArrayList<>(cacheNames.size());
 
         for (String cacheName : cacheNames) {
-            DynamicCacheChangeRequest req = DynamicCacheChangeRequest.stopRequest(ctx, cacheName, true);
+            DynamicCacheChangeRequest req = DynamicCacheChangeRequest.stopRequest(ctx, cacheName, false, true);
 
             reqs.add(req);
         }
@@ -2473,7 +2504,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         List<DynamicCacheChangeRequest> reqs = new ArrayList<>();
 
         for (String cacheName : cacheNames()) {
-            DynamicCacheChangeRequest req = DynamicCacheChangeRequest.stopRequest(ctx, cacheName, false);
+            DynamicCacheChangeRequest req = DynamicCacheChangeRequest.stopRequest(ctx, cacheName, false, false);
 
             reqs.add(req);
         }
@@ -3366,6 +3397,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      * @param cacheName Cache name
      * @param nearCfg Near cache configuration
      * @param cacheType Cache type
+     * @param sql Whether the cache needs to be created as the result of SQL {@code CREATE TABLE} command.
      * @param failIfExists Fail if exists flag.
      * @param failIfNotStarted If {@code true} fails if cache is not started.
      * @return Request or {@code null} if cache already exists.
@@ -3377,6 +3409,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         String cacheName,
         @Nullable NearCacheConfiguration nearCfg,
         CacheType cacheType,
+        boolean sql,
         boolean failIfExists,
         boolean failIfNotStarted
     ) throws IgniteCheckedException {
@@ -3384,6 +3417,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         DynamicCacheChangeRequest req = new DynamicCacheChangeRequest(UUID.randomUUID(), cacheName, ctx.localNodeId());
 
+        req.sql(sql);
+
         req.failIfExists(failIfExists);
 
         if (ccfg != null) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/10a0a84a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
index a7d0d19..78216db 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
@@ -2359,7 +2359,7 @@ public class IgniteCacheProxy<K, V> extends AsyncSupportAdapter<IgniteCache<K, V
         IgniteInternalFuture<?> fut;
 
         try {
-            fut = ctx.kernalContext().cache().dynamicDestroyCache(ctx.name(), true);
+            fut = ctx.kernalContext().cache().dynamicDestroyCache(ctx.name(), false, true);
         }
         finally {
             onLeave(gate);

http://git-wip-us.apache.org/repos/asf/ignite/blob/10a0a84a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
index eb0981b..df9d269 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
@@ -998,6 +998,7 @@ public final class DataStructuresProcessor extends GridProcessorAdapter implemen
                 null,
                 CacheType.INTERNAL,
                 false,
+                false,
                 true,
                 true).get();
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/10a0a84a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index 4187e2b..320c90b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@ -1303,7 +1303,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         // Preserve user specified names as they are.
         newCfg.setSqlEscapeAll(true);
 
-        boolean res = ctx.grid().getOrCreateCache0(newCfg).get2();
+        boolean res = ctx.grid().getOrCreateCache0(newCfg, true).get2();
 
         if (!res && !ifNotExists)
             throw new SchemaOperationException(SchemaOperationException.CODE_TABLE_EXISTS,  entity.getTableName());
@@ -1319,7 +1319,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      */
     @SuppressWarnings("unchecked")
     public void dynamicTableDrop(String cacheName, String tblName, boolean ifExists) throws SchemaOperationException {
-        boolean res = ctx.grid().destroyCache0(cacheName);
+        boolean res = ctx.grid().destroyCache0(cacheName, true);
 
         if (!res && !ifExists)
             throw new SchemaOperationException(SchemaOperationException.CODE_TABLE_NOT_FOUND, tblName);

http://git-wip-us.apache.org/repos/asf/ignite/blob/10a0a84a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java
index 3b9fa69..da260b1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java
@@ -379,7 +379,7 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
             switch (cmd) {
                 case DESTROY_CACHE: {
                     // Do not check thread tx here since there can be active system cache txs.
-                    fut = ((IgniteKernal)ctx.grid()).destroyCacheAsync(cacheName, false).chain(
+                    fut = ((IgniteKernal)ctx.grid()).destroyCacheAsync(cacheName, false, false).chain(
                         new CX1<IgniteInternalFuture<?>, GridRestResponse>() {
                             @Override public GridRestResponse applyx(IgniteInternalFuture<?> f)
                                 throws IgniteCheckedException {

http://git-wip-us.apache.org/repos/asf/ignite/blob/10a0a84a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java
index e7c5ca5..ea3b9e2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java
@@ -198,7 +198,7 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
 
         GridTestUtils.runMultiThreaded(new Callable<Object>() {
             @Override public Object call() throws Exception {
-                futs.add(kernal.context().cache().dynamicDestroyCache(DYNAMIC_CACHE_NAME, true));
+                futs.add(kernal.context().cache().dynamicDestroyCache(DYNAMIC_CACHE_NAME, false, true));
 
                 return null;
             }
@@ -266,7 +266,7 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
             @Override public Object call() throws Exception {
                 IgniteEx kernal = grid(ThreadLocalRandom.current().nextInt(nodeCount()));
 
-                futs.add(kernal.context().cache().dynamicDestroyCache(DYNAMIC_CACHE_NAME, true));
+                futs.add(kernal.context().cache().dynamicDestroyCache(DYNAMIC_CACHE_NAME, false, true));
 
                 return null;
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/10a0a84a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsIgniteMock.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsIgniteMock.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsIgniteMock.java
index ed0af3c..b4fcc70 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsIgniteMock.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsIgniteMock.java
@@ -323,14 +323,14 @@ public class IgfsIgniteMock implements IgniteEx {
 
     /** {@inheritDoc} */
     @Override public <K, V> IgniteBiTuple<IgniteCache<K, V>, Boolean> getOrCreateCache0(
-        CacheConfiguration<K, V> cacheCfg) {
+        CacheConfiguration<K, V> cacheCfg, boolean sql) {
         throwUnsupported();
 
         return null;
     }
 
     /** {@inheritDoc} */
-    @Override public boolean destroyCache0(String cacheName) throws CacheException {
+    @Override public boolean destroyCache0(String cacheName, boolean sql) throws CacheException {
         throwUnsupported();
 
         return false;

http://git-wip-us.apache.org/repos/asf/ignite/blob/10a0a84a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
index f459e57..89ab282 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
@@ -536,7 +536,7 @@ public class IgniteProcessProxy implements IgniteEx {
 
     /** {@inheritDoc} */
     @Override public <K, V> IgniteBiTuple<IgniteCache<K, V>, Boolean> getOrCreateCache0(
-        CacheConfiguration<K, V> cacheCfg) {
+        CacheConfiguration<K, V> cacheCfg, boolean sql) {
         throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
@@ -546,7 +546,7 @@ public class IgniteProcessProxy implements IgniteEx {
     }
 
     /** {@inheritDoc} */
-    @Override public boolean destroyCache0(String cacheName) throws CacheException {
+    @Override public boolean destroyCache0(String cacheName, boolean sql) throws CacheException {
         throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/10a0a84a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicTableSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicTableSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicTableSelfTest.java
index b072fdb..4d74bac 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicTableSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicTableSelfTest.java
@@ -23,7 +23,9 @@ import java.util.Collections;
 import java.util.List;
 import java.util.concurrent.Callable;
 
+import javax.cache.CacheException;
 import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteException;
 import org.apache.ignite.Ignition;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheMode;
@@ -34,6 +36,7 @@ import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.binary.BinaryMarshaller;
+import org.apache.ignite.internal.processors.cache.DynamicCacheDescriptor;
 import org.apache.ignite.internal.processors.query.GridQueryProperty;
 import org.apache.ignite.internal.processors.query.IgniteSQLException;
 import org.apache.ignite.internal.processors.query.QueryTypeDescriptorImpl;
@@ -59,8 +62,6 @@ public class H2DynamicTableSelfTest extends AbstractSchemaSelfTest {
         for (IgniteConfiguration cfg : configurations())
             Ignition.start(cfg);
 
-        client().getOrCreateCache(cacheConfigurationForIndexing());
-
         client().addCacheConfiguration(cacheConfiguration());
     }
 
@@ -72,8 +73,16 @@ public class H2DynamicTableSelfTest extends AbstractSchemaSelfTest {
     }
 
     /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        client().getOrCreateCache(cacheConfigurationForIndexing());
+    }
+
+    /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
-        client().destroyCache("Person");
+        if (client().cache("Person") != null)
+            cache().query(new SqlFieldsQuery("DROP TABLE IF EXISTS \"Person\".\"Person\""));
 
         super.afterTest();
     }
@@ -92,6 +101,12 @@ public class H2DynamicTableSelfTest extends AbstractSchemaSelfTest {
 
             assertNotNull(node.cache("Person"));
 
+            DynamicCacheDescriptor cacheDesc = node.context().cache().cacheDescriptor("Person");
+
+            assertNotNull(cacheDesc);
+
+            assertTrue(cacheDesc.sql());
+
             QueryTypeDescriptorImpl desc = typeExisting(node, "Person", "Person");
 
             assertEquals(Object.class, desc.keyClass());
@@ -142,6 +157,7 @@ public class H2DynamicTableSelfTest extends AbstractSchemaSelfTest {
      * Test that attempting to {@code CREATE TABLE} that already exists yields an error.
      * @throws Exception if failed.
      */
+    @SuppressWarnings("ThrowableResultOfMethodCallIgnored")
     public void testCreateExistingTable() throws Exception {
         cache().query(new SqlFieldsQuery("CREATE TABLE \"Person\" (\"id\" int, \"city\" varchar," +
             " \"name\" varchar, \"surname\" varchar, \"age\" int, PRIMARY KEY (\"id\", \"city\")) WITH " +
@@ -193,6 +209,7 @@ public class H2DynamicTableSelfTest extends AbstractSchemaSelfTest {
      * Test that attempting to {@code DROP TABLE} that does not exist yields an error.
      * @throws Exception if failed.
      */
+    @SuppressWarnings("ThrowableResultOfMethodCallIgnored")
     public void testDropMissingTable() throws Exception {
         GridTestUtils.assertThrows(null, new Callable<Object>() {
             @Override public Object call() throws Exception {
@@ -204,6 +221,62 @@ public class H2DynamicTableSelfTest extends AbstractSchemaSelfTest {
     }
 
     /**
+     * Check that {@code DROP TABLE} for caches not created with {@code CREATE TABLE} yields an error.
+     * @throws Exception if failed.
+     */
+    @SuppressWarnings("ThrowableResultOfMethodCallIgnored")
+    public void testDropNonDynamicTable() throws Exception {
+        GridTestUtils.assertThrows(null, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                cache().query(new SqlFieldsQuery("DROP TABLE \"Integer\""));
+
+                return null;
+            }
+        }, IgniteSQLException.class,
+        "Only cache created with CREATE TABLE may be removed with DROP TABLE [cacheName=cache_idx]");
+    }
+
+    /**
+     * Test that attempting to destroy via cache API a cache created via SQL yields an error.
+     * @throws Exception if failed.
+     */
+    @SuppressWarnings("ThrowableResultOfMethodCallIgnored")
+    public void testDestroyDynamicSqlCache() throws Exception {
+        cache().query(new SqlFieldsQuery("CREATE TABLE \"Person\" (\"id\" int, \"city\" varchar," +
+            " \"name\" varchar, \"surname\" varchar, \"age\" int, PRIMARY KEY (\"id\", \"city\")) WITH " +
+            "\"cacheTemplate=cache\""));
+
+        GridTestUtils.assertThrows(null, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                client().destroyCache("Person");
+
+                return null;
+            }
+        }, CacheException.class,
+        "Only cache created with cache API may be removed with direct call to destroyCache [cacheName=Person]");
+    }
+
+    /**
+     * Test that attempting to start a node that has a cache with the name already present in the grid and whose
+     * SQL flag does not match that of cache with the same name that is already started, yields an error.
+     * @throws Exception if failed.
+     */
+    @SuppressWarnings("ThrowableResultOfMethodCallIgnored")
+    public void testSqlFlagCompatibilityCheck() throws Exception {
+        cache().query(new SqlFieldsQuery("CREATE TABLE \"Person\" (\"id\" int, \"city\" varchar," +
+            " \"name\" varchar, \"surname\" varchar, \"age\" int, PRIMARY KEY (\"id\", \"city\")) WITH " +
+            "\"cacheTemplate=cache\""));
+
+        GridTestUtils.assertThrows(null, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                Ignition.start(clientConfiguration(5).setCacheConfiguration(new CacheConfiguration("Person")));
+
+                return null;
+            }
+        }, IgniteException.class, "SQL flag mismatch (fix sql flag in cache configuration");
+    }
+
+    /**
      * Check that a property in given descriptor is present and has parameters as expected.
      * @param desc Descriptor.
      * @param name Property name.


[11/19] ignite git commit: IGNITE-5334: Fixed SQL schema handling, so that quoted name is treated as case-sensitive. This closes #2027.

Posted by sb...@apache.org.
IGNITE-5334: Fixed SQL schema handling, so that quoted name is treated as case-sensitive. This closes #2027.


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

Branch: refs/heads/ignite-5075
Commit: b7f20211e372431bf61d661660fbaf8334a919f0
Parents: 64ab5cd
Author: devozerov <vo...@gridgain.com>
Authored: Tue May 30 10:53:49 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Tue May 30 10:53:49 2017 +0300

----------------------------------------------------------------------
 .../processors/query/GridQueryProcessor.java      |  2 +-
 .../internal/processors/query/QueryUtils.java     | 14 +++++++++++---
 .../query/IgniteSqlSchemaIndexingTest.java        | 18 ++++++++++++------
 .../query/IgniteSqlSplitterSelfTest.java          |  2 --
 4 files changed, 24 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b7f20211/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index f3da6d0..4187e2b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@ -677,7 +677,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
                 String cacheName = cctx.name();
 
-                String schemaName = QueryUtils.normalizeSchemaName(cacheName, cctx.config().getSqlSchema(), escape);
+                String schemaName = QueryUtils.normalizeSchemaName(cacheName, cctx.config().getSqlSchema());
 
                 // Prepare candidates.
                 List<Class<?>> mustDeserializeClss = new ArrayList<>();

http://git-wip-us.apache.org/repos/asf/ignite/blob/b7f20211/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
index c327820..b4feea4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
@@ -268,10 +268,11 @@ public class QueryUtils {
      *
      * @param cacheName Cache name.
      * @param schemaName Schema name.
-     * @param escape Whether to perform escape.
      * @return Proper schema name according to ANSI-99 standard.
      */
-    public static String normalizeSchemaName(String cacheName, @Nullable String schemaName, boolean escape) {
+    public static String normalizeSchemaName(String cacheName, @Nullable String schemaName) {
+        boolean escape = false;
+
         String res = schemaName;
 
         if (res == null) {
@@ -282,6 +283,13 @@ public class QueryUtils {
             // SQL synthax perspective. We do not want node to fail on startup due to this.
             escape = true;
         }
+        else {
+            if (res.startsWith("\"") && res.endsWith("\"")) {
+                res = res.substring(1, res.length() - 1);
+
+                escape = true;
+            }
+        }
 
         if (!escape)
             res = normalizeObjectName(res, false);
@@ -308,7 +316,7 @@ public class QueryUtils {
      * Normalize object name.
      *
      * @param str String.
-     * @param replace Whether to perform replace.
+     * @param replace Whether to perform replace of special characters.
      * @return Escaped string.
      */
     public static @Nullable String normalizeObjectName(@Nullable String str, boolean replace) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/b7f20211/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSchemaIndexingTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSchemaIndexingTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSchemaIndexingTest.java
index 4672290..3d994c2 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSchemaIndexingTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSchemaIndexingTest.java
@@ -160,12 +160,12 @@ public class IgniteSqlSchemaIndexingTest extends GridCommonAbstractTest {
             .setSqlEscapeAll(true);
 
         final CacheConfiguration<Integer, Fact> cfgEsc = cacheConfig("escapedSchema", true, Integer.class, Fact.class)
-            .setSqlSchema("SchemaName2")
+            .setSqlSchema("\"SchemaName2\"")
             .setSqlEscapeAll(true);
 
-        escapeCheckSchemaName(ignite(0).createCache(cfg), log, cfg.getSqlSchema());
+        escapeCheckSchemaName(ignite(0).createCache(cfg), log, cfg.getSqlSchema(), false);
 
-        escapeCheckSchemaName(ignite(0).createCache(cfgEsc), log, "SchemaName2");
+        escapeCheckSchemaName(ignite(0).createCache(cfgEsc), log, "SchemaName2", true);
 
         ignite(0).destroyCache(cfg.getName());
         ignite(0).destroyCache(cfgEsc.getName());
@@ -175,9 +175,11 @@ public class IgniteSqlSchemaIndexingTest extends GridCommonAbstractTest {
      * Executes query with and without escaped schema name.
      * @param cache cache for querying
      * @param log logger for assertThrows
-     * @param schemaName - schema name without quotes for testing
+     * @param schemaName Schema name without quotes for testing
+     * @param caseSensitive Whether schema name is case sensitive.
      */
-    private static void escapeCheckSchemaName(final IgniteCache<Integer, Fact> cache, IgniteLogger log, String schemaName) {
+    private static void escapeCheckSchemaName(final IgniteCache<Integer, Fact> cache, IgniteLogger log,
+        String schemaName, boolean caseSensitive) {
         final SqlFieldsQuery qryWrong = new SqlFieldsQuery("select f.id, f.name " +
             "from " + schemaName.toUpperCase() + ".Fact f");
 
@@ -186,12 +188,16 @@ public class IgniteSqlSchemaIndexingTest extends GridCommonAbstractTest {
         GridTestUtils.assertThrows(log, new Callable<Object>() {
             @Override public Object call() throws Exception {
                 cache.query(qryWrong);
+
                 return null;
             }
         }, CacheException.class, "Failed to parse query");
 
+        if (caseSensitive)
+            schemaName = "\"" + schemaName + "\"";
+
         SqlFieldsQuery qryCorrect = new SqlFieldsQuery("select f.\"id\", f.\"name\" " +
-            "from \""+schemaName+"\".\"Fact\" f");
+            "from "+  schemaName + ".\"Fact\" f");
 
         for ( List<?> row : cache.query(qryCorrect)) {
             assertEquals(2, row.size());

http://git-wip-us.apache.org/repos/asf/ignite/blob/b7f20211/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
index 6b984bf..9b5c17a 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
@@ -1125,14 +1125,12 @@ public class IgniteSqlSplitterSelfTest extends GridCommonAbstractTest {
     /**
      */
     public void testSchemaQuoted() {
-        assert false; // TODO test hangs
         doTestSchemaName("\"ppAf\"");
     }
 
     /**
      */
     public void testSchemaQuotedUpper() {
-        assert false; // TODO test hangs
         doTestSchemaName("\"PPAF\"");
     }
 


[03/19] ignite git commit: Fixed DiscoverySpi methods return types.

Posted by sb...@apache.org.
Fixed DiscoverySpi methods return types.


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

Branch: refs/heads/ignite-5075
Commit: 0f8af137caa6d2ab5bd00ea33a34d4a969adff2b
Parents: 6987611
Author: Ижиков Николай Владимирович <ni...@gmail.com>
Authored: Mon May 29 17:23:17 2017 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Mon May 29 17:23:17 2017 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/spi/discovery/DiscoverySpi.java  | 9 +++------
 .../apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java    | 8 ++------
 .../ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java      | 4 ++--
 3 files changed, 7 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/0f8af137/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpi.java
index 2379337..98222a3 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpi.java
@@ -26,7 +26,6 @@ import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.lang.IgniteProductVersion;
 import org.apache.ignite.spi.IgniteSpi;
 import org.apache.ignite.spi.IgniteSpiException;
-import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -89,7 +88,7 @@ public interface DiscoverySpi extends IgniteSpi {
     /**
      * Sets node attributes and node version which will be distributed in grid during
      * join process. Note that these attributes cannot be changed and set only once.
-     *
+     *set
      * @param attrs Map of node attributes.
      * @param ver Product version.
      */
@@ -113,9 +112,8 @@ public interface DiscoverySpi extends IgniteSpi {
      * Sets a handler for initial data exchange between Ignite nodes.
      *
      * @param exchange Discovery data exchange handler.
-     * @return {@code this} for chaining.
      */
-    public TcpDiscoverySpi setDataExchange(DiscoverySpiDataExchange exchange);
+    public void setDataExchange(DiscoverySpiDataExchange exchange);
 
     /**
      * Sets discovery metrics provider. Use metrics provided by
@@ -123,9 +121,8 @@ public interface DiscoverySpi extends IgniteSpi {
      * dynamic metrics between nodes.
      *
      * @param metricsProvider Provider of metrics data.
-     * @return {@code this} for chaining.
      */
-    public TcpDiscoverySpi setMetricsProvider(DiscoveryMetricsProvider metricsProvider);
+    public void setMetricsProvider(DiscoveryMetricsProvider metricsProvider);
 
     /**
      * Tells discovery SPI to disconnect from topology. This is very close to calling

http://git-wip-us.apache.org/repos/asf/ignite/blob/0f8af137/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
index 2a55412..371f308 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
@@ -1311,17 +1311,13 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi {
     }
 
     /** {@inheritDoc} */
-    @Override public TcpDiscoverySpi setDataExchange(DiscoverySpiDataExchange exchange) {
+    @Override public void setDataExchange(DiscoverySpiDataExchange exchange) {
         this.exchange = exchange;
-
-        return this;
     }
 
     /** {@inheritDoc} */
-    @Override public TcpDiscoverySpi setMetricsProvider(DiscoveryMetricsProvider metricsProvider) {
+    @Override public void setMetricsProvider(DiscoveryMetricsProvider metricsProvider) {
         this.metricsProvider = metricsProvider;
-
-        return this;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/0f8af137/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java
index 641c7d1..bf48fcc 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java
@@ -2132,8 +2132,8 @@ public class TcpDiscoverySelfTest extends GridCommonAbstractTest {
         static volatile int marshalledItems;
 
         /** {@inheritDoc} */
-        @Override public TcpDiscoverySpi setDataExchange(final DiscoverySpiDataExchange exchange) {
-            return super.setDataExchange(new DiscoverySpiDataExchange() {
+        @Override public void setDataExchange(final DiscoverySpiDataExchange exchange) {
+            super.setDataExchange(new DiscoverySpiDataExchange() {
                 @Override public DiscoveryDataBag collect(DiscoveryDataBag dataBag) {
                     DiscoveryDataBag bag = exchange.collect(dataBag);
 


[12/19] ignite git commit: IGNITE-5030 Support Spring @Cacheable(sync=true) annotation

Posted by sb...@apache.org.
IGNITE-5030 Support Spring @Cacheable(sync=true) annotation


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

Branch: refs/heads/ignite-5075
Commit: b0e49ad9ad9cba8901b5ffad37b4fe171246a9d0
Parents: b7f2021
Author: mcherkasov <mc...@gridgain.com>
Authored: Tue May 30 14:57:58 2017 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Tue May 30 14:57:58 2017 +0300

----------------------------------------------------------------------
 modules/spring/pom.xml                          |   7 +
 .../apache/ignite/cache/spring/SpringCache.java |  60 +++++++--
 .../ignite/cache/spring/SpringCacheManager.java |  51 ++++++-
 .../GridSpringCacheManagerMultiJvmSelfTest.java | 134 +++++++++++++++++++
 .../GridSpringDynamicCacheTestService.java      |  13 ++
 .../ignite/cache/spring/SpringCacheTest.java    |   2 +-
 .../ignite/cache/spring/spring-caching1.xml     |  56 ++++++++
 .../ignite/cache/spring/spring-caching2.xml     |  56 ++++++++
 8 files changed, 368 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b0e49ad9/modules/spring/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spring/pom.xml b/modules/spring/pom.xml
index edaa013..e739118 100644
--- a/modules/spring/pom.xml
+++ b/modules/spring/pom.xml
@@ -123,6 +123,13 @@
             <type>test-jar</type>
             <scope>test</scope>
         </dependency>
+
+        <dependency>
+            <groupId>com.thoughtworks.xstream</groupId>
+            <artifactId>xstream</artifactId>
+            <version>1.4.8</version>
+            <scope>test</scope>
+        </dependency>
     </dependencies>
 
     <build>

http://git-wip-us.apache.org/repos/asf/ignite/blob/b0e49ad9/modules/spring/src/main/java/org/apache/ignite/cache/spring/SpringCache.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/main/java/org/apache/ignite/cache/spring/SpringCache.java b/modules/spring/src/main/java/org/apache/ignite/cache/spring/SpringCache.java
index f35ed2a..9a8f2a8 100644
--- a/modules/spring/src/main/java/org/apache/ignite/cache/spring/SpringCache.java
+++ b/modules/spring/src/main/java/org/apache/ignite/cache/spring/SpringCache.java
@@ -20,6 +20,7 @@ package org.apache.ignite.cache.spring;
 import java.io.Serializable;
 import java.util.concurrent.Callable;
 import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteLock;
 import org.springframework.cache.Cache;
 import org.springframework.cache.support.SimpleValueWrapper;
 
@@ -33,13 +34,18 @@ class SpringCache implements Cache {
     /** */
     private final IgniteCache<Object, Object> cache;
 
+    /** */
+    private final SpringCacheManager mgr;
+
     /**
      * @param cache Cache.
+     * @param mgr Manager
      */
-    SpringCache(IgniteCache<Object, Object> cache) {
+    SpringCache(IgniteCache<Object, Object> cache, SpringCacheManager mgr) {
         assert cache != null;
 
         this.cache = cache;
+        this.mgr = mgr;
     }
 
     /** {@inheritDoc} */
@@ -75,6 +81,40 @@ class SpringCache implements Cache {
     }
 
     /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public <T> T get(final Object key, final Callable<T> valLdr) {
+        Object val = cache.get(key);
+
+        if (val == null) {
+            IgniteLock lock = mgr.getSyncLock(cache.getName(), key);
+
+            lock.lock();
+
+            try {
+                val = cache.get(key);
+
+                if (val == null) {
+                    try {
+                        T retVal = valLdr.call();
+
+                        val = wrapNull(retVal);
+
+                        cache.put(key, val);
+                    }
+                    catch (Exception e) {
+                        throw new ValueRetrievalException(key, valLdr, e);
+                    }
+                }
+            }
+            finally {
+                lock.unlock();
+            }
+        }
+
+        return (T)unwrapNull(val);
+    }
+
+    /** {@inheritDoc} */
     @Override public void put(Object key, Object val) {
         if (val == null)
             cache.withSkipStore().put(key, NULL);
@@ -83,11 +123,6 @@ class SpringCache implements Cache {
     }
 
     /** {@inheritDoc} */
-    @Override public <T> T get(Object key, Callable<T> valLdr) {
-        throw new UnsupportedOperationException();
-    }
-
-    /** {@inheritDoc} */
     @Override public ValueWrapper putIfAbsent(Object key, Object val) {
         Object old;
 
@@ -116,11 +151,18 @@ class SpringCache implements Cache {
     private static ValueWrapper fromValue(Object val) {
         assert val != null;
 
-        return new SimpleValueWrapper(NULL.equals(val) ? null : val);
+        return new SimpleValueWrapper(unwrapNull(val));
     }
 
-    /**
-     */
+    private static Object unwrapNull(Object val) {
+        return NULL.equals(val) ? null : val;
+    }
+
+    private <T> Object wrapNull(T val) {
+        return val == null ? NULL : val;
+    }
+
+    /** */
     private static class NullValue implements Serializable {
         /** {@inheritDoc} */
         @Override public boolean equals(Object o) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/b0e49ad9/modules/spring/src/main/java/org/apache/ignite/cache/spring/SpringCacheManager.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/main/java/org/apache/ignite/cache/spring/SpringCacheManager.java b/modules/spring/src/main/java/org/apache/ignite/cache/spring/SpringCacheManager.java
index 2a24b42..c04eb26 100644
--- a/modules/spring/src/main/java/org/apache/ignite/cache/spring/SpringCacheManager.java
+++ b/modules/spring/src/main/java/org/apache/ignite/cache/spring/SpringCacheManager.java
@@ -19,8 +19,10 @@ package org.apache.ignite.cache.spring;
 
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Objects;
 import java.util.concurrent.ConcurrentMap;
 import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteLock;
 import org.apache.ignite.IgniteSpring;
 import org.apache.ignite.Ignition;
 import org.apache.ignite.configuration.CacheConfiguration;
@@ -138,6 +140,12 @@ import org.springframework.context.ApplicationContextAware;
  * in caching the data.
  */
 public class SpringCacheManager implements CacheManager, InitializingBean, ApplicationContextAware {
+    /** Default locks count. */
+    private static final int DEFAULT_LOCKS_COUNT = 512;
+
+    /** IgniteLock name prefix. */
+    private static final String SPRING_LOCK_NAME_PREFIX = "springSync";
+
     /** Caches map. */
     private final ConcurrentMap<String, SpringCache> caches = new ConcurrentHashMap8<>();
 
@@ -150,6 +158,9 @@ public class SpringCacheManager implements CacheManager, InitializingBean, Appli
     /** Ignite instance name. */
     private String igniteInstanceName;
 
+    /** Count of IgniteLocks are used for sync get */
+    private int locksCnt = DEFAULT_LOCKS_COUNT;
+
     /** Dynamic cache configuration template. */
     private CacheConfiguration<Object, Object> dynamicCacheCfg;
 
@@ -162,6 +173,9 @@ public class SpringCacheManager implements CacheManager, InitializingBean, Appli
     /** Spring context. */
     private ApplicationContext springCtx;
 
+    /** Locks for value loading to support sync option. */
+    private ConcurrentHashMap8<Integer, IgniteLock> locks = new ConcurrentHashMap8<>();
+
     /** {@inheritDoc} */
     @Override public void setApplicationContext(ApplicationContext ctx) {
         this.springCtx = ctx;
@@ -244,6 +258,22 @@ public class SpringCacheManager implements CacheManager, InitializingBean, Appli
     }
 
     /**
+     * Gets locks count.
+     *
+     * @return locks count.
+     */
+    public int getLocksCount() {
+        return locksCnt;
+    }
+
+    /**
+     * @param locksCnt locks count.
+     */
+    public void setLocksCount(int locksCnt) {
+        this.locksCnt = locksCnt;
+    }
+
+    /**
      * Gets dynamic cache configuration template.
      *
      * @return Dynamic cache configuration template.
@@ -314,7 +344,7 @@ public class SpringCacheManager implements CacheManager, InitializingBean, Appli
             cacheCfg.setName(name);
 
             cache = new SpringCache(nearCacheCfg != null ? ignite.getOrCreateCache(cacheCfg, nearCacheCfg) :
-                ignite.getOrCreateCache(cacheCfg));
+                ignite.getOrCreateCache(cacheCfg), this);
 
             SpringCache old = caches.putIfAbsent(name, cache);
 
@@ -331,4 +361,23 @@ public class SpringCacheManager implements CacheManager, InitializingBean, Appli
 
         return new ArrayList<>(caches.keySet());
     }
+
+    /**
+     * Provides {@link org.apache.ignite.IgniteLock} for specified cache name and key.
+     *
+     * @param name cache name
+     * @param key  key
+     * @return {@link org.apache.ignite.IgniteLock}
+     */
+    IgniteLock getSyncLock(String name, Object key) {
+        int hash = Objects.hash(name, key);
+
+        final int idx = hash % getLocksCount();
+
+        return locks.computeIfAbsent(idx, new ConcurrentHashMap8.Fun<Integer, IgniteLock>() {
+            @Override public IgniteLock apply(Integer integer) {
+                return ignite.reentrantLock(SPRING_LOCK_NAME_PREFIX + idx, true, false, true);
+            }
+        });
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/b0e49ad9/modules/spring/src/test/java/org/apache/ignite/cache/spring/GridSpringCacheManagerMultiJvmSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/java/org/apache/ignite/cache/spring/GridSpringCacheManagerMultiJvmSelfTest.java b/modules/spring/src/test/java/org/apache/ignite/cache/spring/GridSpringCacheManagerMultiJvmSelfTest.java
new file mode 100644
index 0000000..39483fc
--- /dev/null
+++ b/modules/spring/src/test/java/org/apache/ignite/cache/spring/GridSpringCacheManagerMultiJvmSelfTest.java
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.cache.spring;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.CyclicBarrier;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.apache.ignite.testframework.junits.multijvm.IgniteProcessProxy;
+import org.springframework.beans.factory.BeanFactory;
+import org.springframework.context.support.ClassPathXmlApplicationContext;
+
+/**
+ * Spring cache test in multi jvm environment.
+ */
+public class GridSpringCacheManagerMultiJvmSelfTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean isMultiJvm() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getTestIgniteInstanceName(int idx) {
+        return getTestIgniteInstanceName() + idx;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getTestIgniteInstanceName() {
+        return "testGrid";
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSyncCache() throws Exception {
+        IgniteEx loc = startGrid(0);
+
+        final int threads = 4;
+        final int entries = 100_000;
+        final int remoteNum = 2;
+
+        final CountDownLatch latch = new CountDownLatch(1);
+
+        List<IgniteInternalFuture<Integer>> futures = new ArrayList<>(remoteNum);
+
+        for (int i = 0; i < remoteNum; i++) {
+            final int gridIdx = i + 1;
+
+            final IgniteEx remote = startGrid(gridIdx);
+
+            IgniteInternalFuture<Integer> calledCntFut = GridTestUtils.runAsync(new Callable<Integer>() {
+                @Override public Integer call() throws Exception {
+                    latch.await();
+
+                    return executeRemotely((IgniteProcessProxy)remote, new TestIgniteCallable<Integer>() {
+                        @Override public Integer call(Ignite ignite) throws Exception {
+                            BeanFactory factory =
+                                new ClassPathXmlApplicationContext(
+                                    "org/apache/ignite/cache/spring/spring-caching" + gridIdx + ".xml");
+
+                            final GridSpringDynamicCacheTestService dynamicSvc =
+                                (GridSpringDynamicCacheTestService)factory.getBean("dynamicTestService");
+
+                            final CyclicBarrier barrier = new CyclicBarrier(threads);
+
+                            GridTestUtils.runMultiThreaded(
+                                new Callable() {
+                                    @Override public Object call() throws Exception {
+                                        for (int i = 0; i < entries; i++) {
+                                            barrier.await();
+
+                                            assertEquals("value" + i, dynamicSvc.cacheableSync(i));
+                                            assertEquals("value" + i, dynamicSvc.cacheableSync(i));
+                                        }
+
+                                        return null;
+                                    }
+                                },
+                                threads,
+                                "get-sync");
+
+                            return dynamicSvc.called();
+                        }
+                    });
+
+                }
+            });
+
+            futures.add(calledCntFut);
+        }
+
+        latch.countDown();
+
+        int totalCalledCnt = 0;
+
+        for (IgniteInternalFuture<Integer> future : futures)
+            totalCalledCnt += future.get();
+
+        IgniteCache<Object, Object> cache = loc.cache("dynamicCache");
+
+        assertEquals(entries, cache.size());
+        assertEquals(entries, totalCalledCnt);
+
+        for (int i = 0; i < entries; i++)
+            assertEquals("value" + i, cache.get(i));
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b0e49ad9/modules/spring/src/test/java/org/apache/ignite/cache/spring/GridSpringDynamicCacheTestService.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/java/org/apache/ignite/cache/spring/GridSpringDynamicCacheTestService.java b/modules/spring/src/test/java/org/apache/ignite/cache/spring/GridSpringDynamicCacheTestService.java
index 6584277..b15a9c0 100644
--- a/modules/spring/src/test/java/org/apache/ignite/cache/spring/GridSpringDynamicCacheTestService.java
+++ b/modules/spring/src/test/java/org/apache/ignite/cache/spring/GridSpringDynamicCacheTestService.java
@@ -46,6 +46,19 @@ public class GridSpringDynamicCacheTestService {
      * @param key Key.
      * @return Value.
      */
+    @Cacheable(value = "dynamicCache", sync = true)
+    public String cacheableSync(Integer key) {
+        assert key != null;
+
+        cnt.incrementAndGet();
+
+        return "value" + key;
+    }
+
+    /**
+     * @param key Key.
+     * @return Value.
+     */
     @CachePut("dynamicCache")
     public String cachePut(Integer key) {
         assert key != null;

http://git-wip-us.apache.org/repos/asf/ignite/blob/b0e49ad9/modules/spring/src/test/java/org/apache/ignite/cache/spring/SpringCacheTest.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/java/org/apache/ignite/cache/spring/SpringCacheTest.java b/modules/spring/src/test/java/org/apache/ignite/cache/spring/SpringCacheTest.java
index 2324235..06dc263 100644
--- a/modules/spring/src/test/java/org/apache/ignite/cache/spring/SpringCacheTest.java
+++ b/modules/spring/src/test/java/org/apache/ignite/cache/spring/SpringCacheTest.java
@@ -59,7 +59,7 @@ public class SpringCacheTest extends GridCommonAbstractTest {
 
         cacheName = String.valueOf(System.currentTimeMillis());
         nativeCache = ignite.getOrCreateCache(cacheName);
-        springCache = new SpringCache(nativeCache);
+        springCache = new SpringCache(nativeCache, null);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/b0e49ad9/modules/spring/src/test/java/org/apache/ignite/cache/spring/spring-caching1.xml
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/java/org/apache/ignite/cache/spring/spring-caching1.xml b/modules/spring/src/test/java/org/apache/ignite/cache/spring/spring-caching1.xml
new file mode 100644
index 0000000..679fd97
--- /dev/null
+++ b/modules/spring/src/test/java/org/apache/ignite/cache/spring/spring-caching1.xml
@@ -0,0 +1,56 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xmlns:cache="http://www.springframework.org/schema/cache"
+       xsi:schemaLocation="
+        http://www.springframework.org/schema/beans http://www.springframework.org/schema/beans/spring-beans.xsd
+        http://www.springframework.org/schema/cache http://www.springframework.org/schema/cache/spring-cache.xsd">
+    <!--
+        Test service with cacheable methods.
+    -->
+    <bean id="testService" class="org.apache.ignite.cache.spring.GridSpringCacheTestService"/>
+
+    <!--
+        Test service with cacheable methods (dynamic cache).
+    -->
+    <bean id="dynamicTestService" class="org.apache.ignite.cache.spring.GridSpringDynamicCacheTestService"/>
+
+    <!--
+        Cache manager.
+    -->
+    <bean id="cacheManager" class="org.apache.ignite.cache.spring.SpringCacheManager">
+        <property name="igniteInstanceName" value="testGrid1"/>
+        <property name="dynamicCacheConfiguration">
+            <bean class="org.apache.ignite.configuration.CacheConfiguration">
+            </bean>
+        </property>
+    </bean>
+
+    <!--
+        Key generator.
+    -->
+    <bean id="keyGenerator" class="org.apache.ignite.cache.spring.GridSpringCacheTestKeyGenerator"/>
+
+    <!--
+        Enable annotation-driver configuration for caching.
+    -->
+    <cache:annotation-driven key-generator="keyGenerator"/>
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/b0e49ad9/modules/spring/src/test/java/org/apache/ignite/cache/spring/spring-caching2.xml
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/java/org/apache/ignite/cache/spring/spring-caching2.xml b/modules/spring/src/test/java/org/apache/ignite/cache/spring/spring-caching2.xml
new file mode 100644
index 0000000..6a9e25a
--- /dev/null
+++ b/modules/spring/src/test/java/org/apache/ignite/cache/spring/spring-caching2.xml
@@ -0,0 +1,56 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one or more
+  ~ contributor license agreements.  See the NOTICE file distributed with
+  ~ this work for additional information regarding copyright ownership.
+  ~ The ASF licenses this file to You under the Apache License, Version 2.0
+  ~ (the "License"); you may not use this file except in compliance with
+  ~ the License.  You may obtain a copy of the License at
+  ~
+  ~      http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing, software
+  ~ distributed under the License is distributed on an "AS IS" BASIS,
+  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~ See the License for the specific language governing permissions and
+  ~ limitations under the License.
+  -->
+
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xmlns:cache="http://www.springframework.org/schema/cache"
+       xsi:schemaLocation="
+        http://www.springframework.org/schema/beans http://www.springframework.org/schema/beans/spring-beans.xsd
+        http://www.springframework.org/schema/cache http://www.springframework.org/schema/cache/spring-cache.xsd">
+    <!--
+        Test service with cacheable methods.
+    -->
+    <bean id="testService" class="org.apache.ignite.cache.spring.GridSpringCacheTestService"/>
+
+    <!--
+        Test service with cacheable methods (dynamic cache).
+    -->
+    <bean id="dynamicTestService" class="org.apache.ignite.cache.spring.GridSpringDynamicCacheTestService"/>
+
+    <!--
+        Cache manager.
+    -->
+    <bean id="cacheManager" class="org.apache.ignite.cache.spring.SpringCacheManager">
+        <property name="igniteInstanceName" value="testGrid2"/>
+        <property name="dynamicCacheConfiguration">
+            <bean class="org.apache.ignite.configuration.CacheConfiguration">
+            </bean>
+        </property>
+    </bean>
+
+    <!--
+        Key generator.
+    -->
+    <bean id="keyGenerator" class="org.apache.ignite.cache.spring.GridSpringCacheTestKeyGenerator"/>
+
+    <!--
+        Enable annotation-driver configuration for caching.
+    -->
+    <cache:annotation-driven key-generator="keyGenerator"/>
+</beans>


[13/19] ignite git commit: IGNITE-5333 .NET: Include Apache.Ignite.exe in the main NuGet package

Posted by sb...@apache.org.
IGNITE-5333 .NET: Include Apache.Ignite.exe in the main NuGet package

This closes #2028


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

Branch: refs/heads/ignite-5075
Commit: 8237ac6f6b8071dcb18b2a05a429e94cba22e71d
Parents: b0e49ad
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Tue May 30 16:04:28 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Tue May 30 16:04:28 2017 +0300

----------------------------------------------------------------------
 .../StartupTest.cs                              | 78 ++++++++++++++++++++
 .../Apache.Ignite.Core.Tests.NuGet/TestUtil.cs  | 30 ++++++++
 .../Apache.Ignite.Core.nuspec                   |  7 +-
 .../Impl/Common/IgniteHome.cs                   |  3 +-
 .../platforms/dotnet/Apache.Ignite/App.config   | 11 +++
 5 files changed, 125 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/8237ac6f/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/StartupTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/StartupTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/StartupTest.cs
index 5b38bde..db62e09 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/StartupTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/StartupTest.cs
@@ -17,6 +17,11 @@
 
 namespace Apache.Ignite.Core.Tests.NuGet
 {
+    using System;
+    using System.Diagnostics;
+    using System.IO;
+    using System.Linq;
+    using System.Threading;
     using Apache.Ignite.Core.Cache.Configuration;
     using NUnit.Framework;
 
@@ -26,6 +31,24 @@ namespace Apache.Ignite.Core.Tests.NuGet
     public class StartupTest
     {
         /// <summary>
+        /// Tears down the test.
+        /// </summary>
+        [TearDown]
+        public void TearDown()
+        {
+            Ignition.StopAll(true);
+
+            foreach (var proc in Process.GetProcesses())
+            {
+                if (proc.ProcessName.Equals("Apache.Ignite"))
+                {
+                    proc.Kill();
+                    proc.WaitForExit();
+                }
+            }
+        }
+
+        /// <summary>
         /// Tests code configuration.
         /// </summary>
         [Test]
@@ -62,5 +85,60 @@ namespace Apache.Ignite.Core.Tests.NuGet
                 Assert.AreEqual(5, cache[1]);
             }
         }
+
+        /// <summary>
+        /// Tests the executable that is included in NuGet.
+        /// </summary>
+        [Test]
+        public void TestApacheIgniteExe()
+        {
+            var asm = GetType().Assembly;
+            var version = asm.GetName().Version.ToString(3);
+            var packageDirName = "Apache.Ignite." + version + "*";
+            
+            var asmDir = Path.GetDirectoryName(asm.Location);
+            Assert.IsNotNull(asmDir, asmDir);
+
+            var packagesDir = Path.GetFullPath(Path.Combine(asmDir, @"..\..\packages"));
+            Assert.IsTrue(Directory.Exists(packagesDir), packagesDir);
+
+            var packageDir = Directory.GetDirectories(packagesDir, packageDirName).Single();
+            Assert.IsTrue(Directory.Exists(packageDir), packageDir);
+
+            var exePath = Path.Combine(packageDir, @"lib\net40\Apache.Ignite.exe");
+            Assert.IsTrue(File.Exists(exePath), exePath);
+
+            var springPath = Path.GetFullPath(@"config\ignite-config.xml");
+            Assert.IsTrue(File.Exists(springPath), springPath);
+
+            var procInfo = new ProcessStartInfo(exePath, "-springConfigUrl=" + springPath)
+            {
+                CreateNoWindow = true,
+                UseShellExecute = false,
+                RedirectStandardOutput = true,
+                RedirectStandardError = true
+            };
+            
+            var proc = Process.Start(procInfo);
+            Assert.IsNotNull(proc);
+            Assert.IsFalse(proc.HasExited);
+
+            TestUtil.AttachProcessConsoleReader(proc);
+
+            using (var ignite = Ignition.Start(@"config\ignite-config.xml"))
+            {
+                for (var i = 0; i < 100; i++)
+                {
+                    if (ignite.GetCluster().GetNodes().Count == 2)
+                    {
+                        return;
+                    }
+
+                    Thread.Sleep(100);
+                }
+                
+                Assert.Fail("Failed to join to remote node.");
+            }
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/8237ac6f/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/TestUtil.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/TestUtil.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/TestUtil.cs
index 43e5097..2f0a110 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/TestUtil.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/TestUtil.cs
@@ -17,6 +17,10 @@
 
 namespace Apache.Ignite.Core.Tests.NuGet
 {
+    using System;
+    using System.Diagnostics;
+    using System.IO;
+    using System.Threading;
     using Apache.Ignite.Core.Discovery;
     using Apache.Ignite.Core.Discovery.Tcp;
     using Apache.Ignite.Core.Discovery.Tcp.Static;
@@ -39,5 +43,31 @@ namespace Apache.Ignite.Core.Tests.NuGet
                 }
             };
         }
+
+        /// <summary>
+        /// Attaches the process console reader.
+        /// </summary>
+        public static void AttachProcessConsoleReader(Process process)
+        {
+            Attach(process, process.StandardOutput, false);
+            Attach(process, process.StandardError, true);
+        }
+
+        /// <summary>
+        /// Attach output reader to the process.
+        /// </summary>
+        private static void Attach(Process proc, TextReader reader, bool err)
+        {
+            new Thread(() =>
+            {
+                while (!proc.HasExited)
+                {
+                    Console.WriteLine(err ? ">>> {0} ERR: {1}" : ">>> {0} OUT: {1}", proc.Id, reader.ReadLine());
+                }
+            })
+            {
+                IsBackground = true
+            }.Start();
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/8237ac6f/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.nuspec
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.nuspec b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.nuspec
index 8f562f1..7dc9f9a 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.nuspec
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.nuspec
@@ -53,9 +53,10 @@ More info: https://apacheignite-net.readme.io/
         <file src="NuGet\Install.ps1" target="tools" />
         <file src="NuGet\Uninstall.ps1" target="tools" />
         <file src="NuGet\PostBuild.ps1" target="tools" />
-        
-        <!-- Other files should go to Content folder to be automatically included in project. -->
-        <!--<file src="..\..\..\..\config\default-config.xml" target="Content\Config" />-->
+
+        <!-- Include Apache.Ignite.exe -->
+        <file src="..\Apache.Ignite\bin\$configuration$\Apache.Ignite.exe" target="lib\net40" />
+        <file src="..\Apache.Ignite\bin\$configuration$\Apache.Ignite.exe.config" target="lib\net40" />
         
         <!-- 
             Library files (jars) should not be included in project, so that NuGet package restore works properly.

http://git-wip-us.apache.org/repos/asf/ignite/blob/8237ac6f/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/IgniteHome.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/IgniteHome.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/IgniteHome.cs
index 0290095..3d4ad4d 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/IgniteHome.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/IgniteHome.cs
@@ -116,7 +116,8 @@ namespace Apache.Ignite.Core.Impl.Common
                        || // NuGet home
                        (dir.EnumerateDirectories().Any(x => x.Name == "Libs") &&
                         (dir.EnumerateFiles("Apache.Ignite.Core.dll").Any() ||
-                         dir.EnumerateFiles("Apache.Ignite.*.nupkg").Any()));
+                         dir.EnumerateFiles("Apache.Ignite.*.nupkg").Any() ||
+                         dir.EnumerateFiles("Apache.Ignite.nuspec").Any()));
             }
             catch (IOException)
             {

http://git-wip-us.apache.org/repos/asf/ignite/blob/8237ac6f/modules/platforms/dotnet/Apache.Ignite/App.config
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite/App.config b/modules/platforms/dotnet/Apache.Ignite/App.config
index d1d6643..8550529 100644
--- a/modules/platforms/dotnet/Apache.Ignite/App.config
+++ b/modules/platforms/dotnet/Apache.Ignite/App.config
@@ -23,6 +23,10 @@
 -->
 
 <configuration>
+    <configSections>
+        <section name="igniteConfiguration" type="Apache.Ignite.Core.IgniteConfigurationSection, Apache.Ignite.Core" />
+    </configSections>
+    
     <startup>
         <supportedRuntime version="v4.0" sku=".NETFramework,Version=v4.0"/>
     </startup>
@@ -31,7 +35,14 @@
         <gcServer enabled="true" />
     </runtime>
 
+    <igniteConfiguration xmlns="http://ignite.apache.org/schema/dotnet/IgniteConfigurationSection">
+        <!-- Customize Ignite configuration here. -->
+    </igniteConfiguration>
+
     <appSettings>
+        <!-- Use Ignite configuration section defined above. -->
+        <add key="Ignite.ConfigSectionName" value="igniteConfiguration" />
+        
         <!-- Path to spring configuration file relative from IGNITE_HOME (if not provided "config/default-config.xml" is used) -->
         <!-- <add key="Ignite.SpringConfigUrl" value="my-config.xml"/> -->