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

[ignite] branch master updated: IGNITE-12754 .NET: Add thin client service invocation

This is an automated email from the ASF dual-hosted git repository.

ptupitsyn pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ignite.git


The following commit(s) were added to refs/heads/master by this push:
     new f1c0037  IGNITE-12754 .NET: Add thin client service invocation
f1c0037 is described below

commit f1c00372875a3eb13d592fdcf778115a8f2801af
Author: Pavel Tupitsyn <pt...@apache.org>
AuthorDate: Fri Aug 28 17:43:49 2020 +0300

    IGNITE-12754 .NET: Add thin client service invocation
    
    * Implement Services API in .NET Thin Client
    * Fix array handling in `ClientServiceInvokeRequest`
    * Fix binary mode handling in `ClientServiceInvokeRequest` with Java and Platform services
    * Fix array handling in `DelegateConverter` - support `sbyte[]`, `ushort[]`, `uint[], `ulong[]`
    * Fix return type handling in `ServiceProxyTypeGenerator` - support `sbyte`, `ushort`, `uint`, `ulong` and corresponding arrays
    
    Not supported:
    * Timeouts - see IGNITE-13360, current logic is questionable
    * `DateTime` <-> `Timestamp` conversion like in Thick client: unnecessary complexity (requires descriptor retrieval and separate logic for .NET and Java services)
    
    Benchmark:
    ```
    |        Method  |     Mean |    Error |   StdDev |  Gen 0 | Gen 1 | Gen 2 | Allocated |
    |--------------- |---------:|---------:|---------:|-------:|------:|------:|----------:|
    | IntMethodThin  | 31.61 us | 0.627 us | 1.539 us | 0.3052 |     - |     - |   1.85 KB |
    | IntMethodThick | 130.5 us | 27.65 us | 79.33 us | 0.1221 |     - |     - |   1.47 KB |
    ```
---
 .../internal/client/thin/ClientServicesImpl.java   |   2 +-
 .../ignite/internal/client/thin/ClientUtils.java   |  25 +-
 .../client/service/ClientServiceInvokeRequest.java |  24 +-
 .../platform/services/PlatformServices.java        |  48 +-
 .../ignite/platform/PlatformDeployServiceTask.java |  11 +
 .../Apache.Ignite.BenchmarkDotNet/Program.cs       |   2 +-
 .../Services/BenchService.cs}                      |  37 +-
 .../Services/IBenchService.cs}                     |  19 +-
 .../ThinClient/ThinClientServicesBenchmark.cs      |  99 ++++
 .../Apache.Ignite.Core.Tests.csproj                |  11 +
 .../Client/Compute/ComputeClientTests.cs           |   2 +
 .../Client/Services/ITestService.cs                |  66 +++
 .../Client/Services/ITestServiceClient.cs}         |  31 +-
 .../Client/Services/ITestServiceDataTypes.cs       | 133 +++++
 .../Client/Services/ITestServiceGenericMethods.cs} |  22 +-
 .../Client/Services/ITestServiceOverloads.cs}      |  49 +-
 .../Client/Services/ServicesClientTest.cs          | 588 +++++++++++++++++++++
 .../Client/Services/TestService.cs                 | 134 +++++
 .../Client/Services/TestServiceDataTypes.cs        | 258 +++++++++
 .../Client/Services/TestServiceGenericMethods.cs}  |  37 +-
 .../Client/Services/TestServiceOverloads.cs        | 106 ++++
 .../IgniteStartStopTest.cs                         |   2 +-
 .../Apache.Ignite.Core.Tests/ProjectFilesTest.cs   |  66 ++-
 .../Services/IJavaService.cs                       | 169 ++++++
 .../Services/ServicesTest.cs                       | 165 +-----
 .../dotnet/Apache.Ignite.Core.Tests/TestUtils.cs   |  17 +
 .../Apache.Ignite.Core/Apache.Ignite.Core.csproj   |   2 +
 .../Client/Compute/IComputeClient.cs               |  13 +-
 .../Client/IClientClusterGroup.cs                  |  17 +-
 .../Apache.Ignite.Core/Client/IIgniteClient.cs     |   8 +-
 .../Client/Services/IServicesClient.cs             |  55 ++
 .../Apache.Ignite.Core/Impl/Client/ClientOp.cs     |   5 +-
 .../Impl/Client/Cluster/ClientClusterGroup.cs      |  16 +-
 .../Impl/Client/Compute/ComputeClient.cs           |   6 +
 .../Apache.Ignite.Core/Impl/Client/IgniteClient.cs |  13 +
 .../Impl/Client/Services/ServicesClient.cs         | 157 ++++++
 .../Impl/Common/DelegateConverter.cs               | 143 ++++-
 .../Impl/Services/ServiceProxyTypeGenerator.cs     |  51 +-
 38 files changed, 2301 insertions(+), 308 deletions(-)

diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientServicesImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientServicesImpl.java
index 5bf5234..7f83606 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientServicesImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientServicesImpl.java
@@ -121,7 +121,7 @@ class ClientServicesImpl implements ClientServices {
 
                 return ch.service(ClientOperation.SERVICE_INVOKE,
                     req -> writeServiceInvokeRequest(req, nodeIds, method, args),
-                    res -> utils.readObject(res.in(), false)
+                    res -> utils.readObject(res.in(), false, method.getReturnType())
                 );
             }
             catch (ClientError e) {
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientUtils.java
index d59200f..fe866b8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientUtils.java
@@ -538,19 +538,24 @@ final class ClientUtils {
 
     /** Read Ignite binary object from input stream. */
     <T> T readObject(BinaryInputStream in, boolean keepBinary) {
+        return readObject(in, keepBinary, null);
+    }
+
+    /** Read Ignite binary object from input stream. */
+    <T> T readObject(BinaryInputStream in, boolean keepBinary, Class<T> clazz) {
         if (keepBinary)
             return (T)marsh.unmarshal(in);
         else {
             BinaryReaderHandles hnds = new BinaryReaderHandles();
 
-            return (T)unwrapBinary(marsh.deserialize(in, hnds), hnds);
+            return (T)unwrapBinary(marsh.deserialize(in, hnds), hnds, clazz);
         }
     }
 
     /**
      * Unwrap binary object.
      */
-    private Object unwrapBinary(Object obj, BinaryReaderHandles hnds) {
+    private Object unwrapBinary(Object obj, BinaryReaderHandles hnds, Class<?> clazz) {
         if (obj instanceof BinaryObjectImpl) {
             BinaryObjectImpl obj0 = (BinaryObjectImpl)obj;
 
@@ -563,7 +568,7 @@ final class ClientUtils {
         else if (BinaryUtils.knownMap(obj))
             return unwrapMap((Map<Object, Object>)obj, hnds);
         else if (obj instanceof Object[])
-            return unwrapArray((Object[])obj, hnds);
+            return unwrapArray((Object[])obj, hnds, clazz);
         else
             return obj;
     }
@@ -575,7 +580,7 @@ final class ClientUtils {
         Collection<Object> col0 = BinaryUtils.newKnownCollection(col);
 
         for (Object obj0 : col)
-            col0.add(unwrapBinary(obj0, hnds));
+            col0.add(unwrapBinary(obj0, hnds, null));
 
         return (col0 instanceof MutableSingletonList) ? U.convertToSingletonList(col0) : col0;
     }
@@ -587,7 +592,7 @@ final class ClientUtils {
         Map<Object, Object> map0 = BinaryUtils.newMap(map);
 
         for (Map.Entry<Object, Object> e : map.entrySet())
-            map0.put(unwrapBinary(e.getKey(), hnds), unwrapBinary(e.getValue(), hnds));
+            map0.put(unwrapBinary(e.getKey(), hnds, null), unwrapBinary(e.getValue(), hnds, null));
 
         return map0;
     }
@@ -595,14 +600,18 @@ final class ClientUtils {
     /**
      * Unwrap array with binary objects.
      */
-    private Object[] unwrapArray(Object[] arr, BinaryReaderHandles hnds) {
+    private Object[] unwrapArray(Object[] arr, BinaryReaderHandles hnds, Class<?> arrayClass) {
         if (BinaryUtils.knownArray(arr))
             return arr;
 
-        Object[] res = (Object[])Array.newInstance(arr.getClass().getComponentType(), arr.length);
+        Class<?> componentType = arrayClass != null && arrayClass.isArray()
+                ? arrayClass.getComponentType()
+                : arr.getClass().getComponentType();
+
+        Object[] res = (Object[])Array.newInstance(componentType, arr.length);
 
         for (int i = 0; i < arr.length; i++)
-            res[i] = unwrapBinary(arr[i], hnds);
+            res[i] = unwrapBinary(arr[i], hnds, null);
 
         return res;
     }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/service/ClientServiceInvokeRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/service/ClientServiceInvokeRequest.java
index e51780b..456d077 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/service/ClientServiceInvokeRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/service/ClientServiceInvokeRequest.java
@@ -150,29 +150,33 @@ public class ClientServiceInvokeRequest extends ClientRequest {
 
         IgniteServices services = grp.services();
 
-        if (!keepBinary() && args.length > 0) {
-            for (int i = 0; i < args.length; i++) {
-                if (paramTypeIds != null)
-                    reader.readInt(); // Skip parameter typeId, we already read it in constructor.
-
-                args[i] = reader.readObject();
-            }
-        }
-
         try {
             Object res;
 
             if (PlatformService.class.isAssignableFrom(svcCls)) {
+                // Never deserialize platform service arguments and result: may contain platform-only types.
                 PlatformService proxy = services.serviceProxy(name, PlatformService.class, false, timeout);
 
-                res = proxy.invokeMethod(methodName, keepBinary(), !keepBinary(), args);
+                res = proxy.invokeMethod(methodName, keepBinary(), false, args);
             }
             else {
+                // Deserialize Java service arguments when not in keepBinary mode.
+                if (!keepBinary() && args.length > 0) {
+                    for (int i = 0; i < args.length; i++) {
+                        if (paramTypeIds != null)
+                            reader.readInt(); // Skip parameter typeId, we already read it in constructor.
+
+                        args[i] = reader.readObject();
+                    }
+                }
+
                 GridServiceProxy<?> proxy = new GridServiceProxy<>(grp, name, Service.class, false, timeout,
                     ctx.kernalContext());
 
                 Method method = resolveMethod(ctx, svcCls);
 
+                PlatformServices.convertArrayArgs(args, method);
+
                 res = proxy.invokeMethod(method, args);
             }
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java
index 8e67cab..21c3d72 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java
@@ -527,6 +527,33 @@ public class PlatformServices extends PlatformAbstractTarget {
     }
 
     /**
+     * Convert Object[] to T[] when required:
+     * Ignite loses array item types when passing arguments through GridServiceProxy.
+     *
+     * @param args Service method args.
+     * @param mtd Target method.
+     */
+    public static void convertArrayArgs(Object[] args, Method mtd) {
+        for (int i = 0; i < args.length; i++) {
+            Object arg = args[i];
+
+            if (arg instanceof Object[]) {
+                Class<?> parameterType = mtd.getParameterTypes()[i];
+
+                if (parameterType.isArray() && parameterType != Object[].class) {
+                    Object[] arr = (Object[])arg;
+                    Object newArg = Array.newInstance(parameterType.getComponentType(), arr.length);
+
+                    for (int j = 0; j < arr.length; j++)
+                        Array.set(newArg, j, arr[j]);
+
+                    args[i] = newArg;
+                }
+            }
+        }
+    }
+
+    /**
      * Proxy holder.
      */
     @SuppressWarnings({"unchecked", "rawtypes"})
@@ -593,26 +620,7 @@ public class PlatformServices extends PlatformAbstractTarget {
                     args = PlatformUtils.unwrapBinariesInArray(args);
 
                 Method mtd = getMethod(serviceClass, mthdName, args);
-
-                // Convert Object[] to T[] when required:
-                // Ignite loses array item types when passing arguments through GridServiceProxy.
-                for (int i = 0; i < args.length; i++) {
-                    Object arg = args[i];
-
-                    if (arg instanceof Object[]) {
-                        Class<?> parameterType = mtd.getParameterTypes()[i];
-
-                        if (parameterType.isArray() && parameterType != Object[].class) {
-                            Object[] arr = (Object[])arg;
-                            Object newArg = Array.newInstance(parameterType.getComponentType(), arr.length);
-
-                            for (int j = 0; j < arr.length; j++)
-                                Array.set(newArg, j, arr[j]);
-
-                            args[i] = newArg;
-                        }
-                    }
-                }
+                convertArrayArgs(args, mtd);
 
                 try {
                     return ((GridServiceProxy)proxy).invokeMethod(mtd, args);
diff --git a/modules/core/src/test/java/org/apache/ignite/platform/PlatformDeployServiceTask.java b/modules/core/src/test/java/org/apache/ignite/platform/PlatformDeployServiceTask.java
index 6986ae4..c781be8 100644
--- a/modules/core/src/test/java/org/apache/ignite/platform/PlatformDeployServiceTask.java
+++ b/modules/core/src/test/java/org/apache/ignite/platform/PlatformDeployServiceTask.java
@@ -33,6 +33,7 @@ import org.apache.ignite.compute.ComputeJobAdapter;
 import org.apache.ignite.compute.ComputeJobResult;
 import org.apache.ignite.compute.ComputeTaskAdapter;
 import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.resources.IgniteInstanceResource;
 import org.apache.ignite.services.Service;
 import org.apache.ignite.services.ServiceContext;
@@ -412,5 +413,15 @@ public class PlatformDeployServiceTask extends ComputeTaskAdapter<String, Object
 
             return o.toBuilder().setField("field", 15).build();
         }
+
+        /** */
+        public void sleep(long delayMs) {
+            try {
+                U.sleep(delayMs);
+            }
+            catch (Exception e) {
+                throw new IgniteException(e);
+            }
+        }
     }
 }
diff --git a/modules/platforms/dotnet/Apache.Ignite.BenchmarkDotNet/Program.cs b/modules/platforms/dotnet/Apache.Ignite.BenchmarkDotNet/Program.cs
index 624f720..d9f53d4 100644
--- a/modules/platforms/dotnet/Apache.Ignite.BenchmarkDotNet/Program.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.BenchmarkDotNet/Program.cs
@@ -30,7 +30,7 @@ namespace Apache.Ignite.BenchmarkDotNet
         /// </summary>
         public static void Main()
         {
-            BenchmarkRunner.Run<ThinClientCachePutBenchmark>();
+            BenchmarkRunner.Run<ThinClientServicesBenchmark>();
         }
     }
 }
diff --git a/modules/platforms/dotnet/Apache.Ignite.BenchmarkDotNet/Program.cs b/modules/platforms/dotnet/Apache.Ignite.BenchmarkDotNet/ThinClient/Services/BenchService.cs
similarity index 57%
copy from modules/platforms/dotnet/Apache.Ignite.BenchmarkDotNet/Program.cs
copy to modules/platforms/dotnet/Apache.Ignite.BenchmarkDotNet/ThinClient/Services/BenchService.cs
index 624f720..d5fed7a 100644
--- a/modules/platforms/dotnet/Apache.Ignite.BenchmarkDotNet/Program.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.BenchmarkDotNet/ThinClient/Services/BenchService.cs
@@ -1,4 +1,4 @@
-/*
+/*
  * 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.
@@ -15,22 +15,37 @@
  * limitations under the License.
  */
 
-namespace Apache.Ignite.BenchmarkDotNet
+namespace Apache.Ignite.BenchmarkDotNet.ThinClient.Services
 {
-    using Apache.Ignite.BenchmarkDotNet.ThinClient;
-    using global::BenchmarkDotNet.Running;
+    using Apache.Ignite.Core.Services;
 
     /// <summary>
-    /// Benchmark runner.
+    /// Benchmark service.
     /// </summary>
-    public static class Program
+    public class BenchService : IService, IBenchService
     {
-        /// <summary>
-        /// Main.
-        /// </summary>
-        public static void Main()
+        /** <inheritdoc /> */
+        public int Add(int x, int y)
         {
-            BenchmarkRunner.Run<ThinClientCachePutBenchmark>();
+            return x + y;
+        }
+
+        /** <inheritdoc /> */
+        public void Init(IServiceContext context)
+        {
+            // No-op.
+        }
+
+        /** <inheritdoc /> */
+        public void Execute(IServiceContext context)
+        {
+            // No-op.
+        }
+
+        /** <inheritdoc /> */
+        public void Cancel(IServiceContext context)
+        {
+            // No-op.
         }
     }
 }
diff --git a/modules/platforms/dotnet/Apache.Ignite.BenchmarkDotNet/Program.cs b/modules/platforms/dotnet/Apache.Ignite.BenchmarkDotNet/ThinClient/Services/IBenchService.cs
similarity index 69%
copy from modules/platforms/dotnet/Apache.Ignite.BenchmarkDotNet/Program.cs
copy to modules/platforms/dotnet/Apache.Ignite.BenchmarkDotNet/ThinClient/Services/IBenchService.cs
index 624f720..5f780ae 100644
--- a/modules/platforms/dotnet/Apache.Ignite.BenchmarkDotNet/Program.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.BenchmarkDotNet/ThinClient/Services/IBenchService.cs
@@ -1,4 +1,4 @@
-/*
+/*
  * 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.
@@ -15,22 +15,13 @@
  * limitations under the License.
  */
 
-namespace Apache.Ignite.BenchmarkDotNet
+namespace Apache.Ignite.BenchmarkDotNet.ThinClient.Services
 {
-    using Apache.Ignite.BenchmarkDotNet.ThinClient;
-    using global::BenchmarkDotNet.Running;
-
     /// <summary>
-    /// Benchmark runner.
+    /// Benchmark service interface.
     /// </summary>
-    public static class Program
+    public interface IBenchService
     {
-        /// <summary>
-        /// Main.
-        /// </summary>
-        public static void Main()
-        {
-            BenchmarkRunner.Run<ThinClientCachePutBenchmark>();
-        }
+        int Add(int x, int y);
     }
 }
diff --git a/modules/platforms/dotnet/Apache.Ignite.BenchmarkDotNet/ThinClient/ThinClientServicesBenchmark.cs b/modules/platforms/dotnet/Apache.Ignite.BenchmarkDotNet/ThinClient/ThinClientServicesBenchmark.cs
new file mode 100644
index 0000000..431e8e3
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.BenchmarkDotNet/ThinClient/ThinClientServicesBenchmark.cs
@@ -0,0 +1,99 @@
+/*
+ * 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.BenchmarkDotNet.ThinClient
+{
+    using System;
+    using Apache.Ignite.BenchmarkDotNet.ThinClient.Services;
+    using Apache.Ignite.Core;
+    using global::BenchmarkDotNet.Attributes;
+
+    /// <summary>
+    /// Thin client services benchmark.
+    /// <para />
+    /// |        Method  |     Mean |    Error |   StdDev |  Gen 0 | Gen 1 | Gen 2 | Allocated |
+    /// |--------------- |---------:|---------:|---------:|-------:|------:|------:|----------:|
+    /// | IntMethodThin  | 31.61 us | 0.627 us | 1.539 us | 0.3052 |     - |     - |   1.85 KB |
+    /// | IntMethodThick | 130.5 us | 27.65 us | 79.33 us | 0.1221 |     - |     - |   1.47 KB |
+    /// </summary>
+    [MemoryDiagnoser]
+    public class ThinClientServicesBenchmark : ThinClientBenchmarkBase
+    {
+        /** */
+        private const string ServiceName = nameof(BenchService);
+
+        /** */
+        private IIgnite ThickClient { get; set; }
+
+        /** */
+        private IBenchService ThickService { get; set; }
+
+        /** */
+        private IBenchService ThinService { get; set; }
+
+        /** <inheritdoc /> */
+        public override void GlobalSetup()
+        {
+            base.GlobalSetup();
+
+            var services = Ignite.GetServices();
+            services.DeployClusterSingleton(ServiceName, new BenchService());
+
+            var clientCfg = new IgniteConfiguration(Utils.GetIgniteConfiguration())
+            {
+                ClientMode = true,
+                IgniteInstanceName = "Client"
+            };
+
+            ThickClient = Ignition.Start(clientCfg);
+
+            ThickService = ThickClient.GetServices().GetServiceProxy<IBenchService>(ServiceName);
+            ThinService = Client.GetServices().GetServiceProxy<IBenchService>(ServiceName);
+        }
+
+        /** <inheritdoc /> */
+        public override void GlobalCleanup()
+        {
+            ThickClient.Dispose();
+            base.GlobalCleanup();
+        }
+
+        /** */
+        [Benchmark]
+        public void IntMethodThin()
+        {
+            var res = ThinService.Add(2, 3);
+
+            if (res != 5)
+            {
+                throw new Exception();
+            }
+        }
+
+        /** */
+        [Benchmark]
+        public void IntMethodThick()
+        {
+            var res = ThickService.Add(2, 3);
+
+            if (res != 5)
+            {
+                throw new Exception();
+            }
+        }
+    }
+}
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 c92cd0a..e4b27a9 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
@@ -188,6 +188,16 @@
     <Compile Include="Client\RawSocketTest.cs" />
     <Compile Include="Client\ClientConnectionTest.cs" />
     <Compile Include="Client\IgniteClientConfigurationTest.cs" />
+    <Compile Include="Client\Services\ITestService.cs" />
+    <Compile Include="Client\Services\ITestServiceClient.cs" />
+    <Compile Include="Client\Services\ITestServiceDataTypes.cs" />
+    <Compile Include="Client\Services\ITestServiceGenericMethods.cs" />
+    <Compile Include="Client\Services\ITestServiceOverloads.cs" />
+    <Compile Include="Client\Services\ServicesClientTest.cs" />
+    <Compile Include="Client\Services\TestService.cs" />
+    <Compile Include="Client\Services\TestServiceDataTypes.cs" />
+    <Compile Include="Client\Services\TestServiceGenericMethods.cs" />
+    <Compile Include="Client\Services\TestServiceOverloads.cs" />
     <Compile Include="Common\IgniteProductVersionTests.cs" />
     <Compile Include="Compute\ComputeApiTest.JavaTask.cs" />
     <Compile Include="Compute\ComputeWithExecutorTest.cs" />
@@ -263,6 +273,7 @@
     <Compile Include="Plugin\TestIgnitePluginConfiguration.cs" />
     <Compile Include="Plugin\TestIgnitePluginException.cs" />
     <Compile Include="Plugin\TestIgnitePluginProvider.cs" />
+    <Compile Include="Services\IJavaService.cs" />
     <Compile Include="Ssl\SslConfigurationTest.cs" />
     <Compile Include="TaskExtensions.cs" />
     <Compile Include="TestAppConfig.cs" />
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Compute/ComputeClientTests.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Compute/ComputeClientTests.cs
index f46468e..b24ee58 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Compute/ComputeClientTests.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Compute/ComputeClientTests.cs
@@ -315,12 +315,14 @@ namespace Apache.Ignite.Core.Tests.Client.Compute
             var nodeIds = Client.GetCluster().GetNodes().Select(n => n.Id).ToArray();
 
             CollectionAssert.AreEquivalent(nodeIds, getProjection(Client.GetCompute()));
+            Assert.AreSame(Client.GetCluster(), Client.GetCompute().ClusterGroup);
 
             // One node.
             var nodeId = nodeIds[1];
             var proj = Client.GetCluster().ForPredicate(n => n.Id == nodeId);
 
             Assert.AreEqual(new[]{nodeId}, getProjection(proj.GetCompute()));
+            Assert.AreEqual(new[]{nodeId}, proj.GetCompute().ClusterGroup.GetNodes().Select(n => n.Id));
 
             // Two nodes.
             proj = Client.GetCluster().ForPredicate(n => n.Id != nodeId);
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Services/ITestService.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Services/ITestService.cs
new file mode 100644
index 0000000..4bfba68
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Services/ITestService.cs
@@ -0,0 +1,66 @@
+/*
+ * 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.Client.Services
+{
+    using System;
+    using System.Threading.Tasks;
+    using Apache.Ignite.Core.Binary;
+    using Apache.Ignite.Core.Tests.Client.Cache;
+
+    /// <summary>
+    /// Test service interface.
+    /// </summary>
+    public interface ITestService
+    {
+        /** */
+        int IntProperty { get; set; }
+
+        /** */
+        Person PersonProperty { get; set; }
+
+        /** */
+        void VoidMethod();
+
+        /** */
+        int IntMethod();
+
+        /** */
+        void ExceptionalMethod();
+
+        /** */
+        Task<int> AsyncMethod();
+
+        /** */
+        Person PersonMethod(Person person);
+
+        /** */
+        IBinaryObject PersonMethodBinary(IBinaryObject person);
+
+        /** */
+        Person[] PersonArrayMethod(Person[] persons);
+
+        /** */
+        IBinaryObject[] PersonArrayMethodBinary(IBinaryObject[] persons);
+
+        /** */
+        void Sleep(TimeSpan delay);
+
+        /** */
+        Guid GetNodeId();
+    }
+}
diff --git a/modules/platforms/dotnet/Apache.Ignite.BenchmarkDotNet/Program.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Services/ITestServiceClient.cs
similarity index 52%
copy from modules/platforms/dotnet/Apache.Ignite.BenchmarkDotNet/Program.cs
copy to modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Services/ITestServiceClient.cs
index 624f720..50d409f 100644
--- a/modules/platforms/dotnet/Apache.Ignite.BenchmarkDotNet/Program.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Services/ITestServiceClient.cs
@@ -1,4 +1,4 @@
-/*
+/*
  * 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.
@@ -15,22 +15,29 @@
  * limitations under the License.
  */
 
-namespace Apache.Ignite.BenchmarkDotNet
+namespace Apache.Ignite.Core.Tests.Client.Services
 {
-    using Apache.Ignite.BenchmarkDotNet.ThinClient;
-    using global::BenchmarkDotNet.Running;
+    using Apache.Ignite.Core.Binary;
+    using Apache.Ignite.Core.Tests.Client.Cache;
 
     /// <summary>
-    /// Benchmark runner.
+    /// Client-side counterpart for <see cref="ITestService"/>.
+    /// <para />
+    /// Clients can use a different service interface. Method signatures can also differ.
     /// </summary>
-    public static class Program
+    public interface ITestServiceClient
     {
         /// <summary>
-        /// Main.
+        /// Counterpart for <see cref="ITestService.PersonMethodBinary"/>.
+        /// <para />
+        /// Client-side interface operates on <see cref="Person"/>,
+        /// but server-side method uses <see cref="IBinaryObject"/>: 
         /// </summary>
-        public static void Main()
-        {
-            BenchmarkRunner.Run<ThinClientCachePutBenchmark>();
-        }
+        Person PersonMethodBinary(Person person);
+        
+        /// <summary>
+        /// Counterpart for <see cref="ITestService.PersonMethod"/>.
+        /// </summary>
+        IBinaryObject PersonMethod(IBinaryObject person);
     }
-}
+}
\ No newline at end of file
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Services/ITestServiceDataTypes.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Services/ITestServiceDataTypes.cs
new file mode 100644
index 0000000..761bfaf
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Services/ITestServiceDataTypes.cs
@@ -0,0 +1,133 @@
+/*
+ * 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.Client.Services
+{
+    using System;
+    using System.Collections.Generic;
+
+    /// <summary>
+    /// Service interface for testing various data types passing.
+    /// </summary>
+    public interface ITestServiceDataTypes
+    {
+        /** */
+        byte GetByte(byte x);
+
+        /** */
+        byte[] GetByteArray(byte[] x);
+
+        /** */
+        sbyte GetSbyte(sbyte x);
+
+        /** */
+        sbyte[] GetSbyteArray(sbyte[] x);
+
+        /** */
+        char GetChar(char x);
+
+        /** */
+        char[] GetCharArray(char[] x);
+
+        /** */
+        short GetShort(short x);
+
+        /** */
+        short[] GetShortArray(short[] x);
+
+        /** */
+        ushort GetUShort(ushort x);
+
+        /** */
+        ushort[] GetUShortArray(ushort[] x);
+
+        /** */
+        int GetInt(int x);
+
+        /** */
+        int[] GetIntArray(int[] x);
+
+        /** */
+        uint GetUInt(uint x);
+
+        /** */
+        uint[] GetUIntArray(uint[] x);
+
+        /** */
+        long GetLong(long x);
+
+        /** */
+        long[] GetLongArray(long[] x);
+
+        /** */
+        ulong GetULong(ulong x);
+
+        /** */
+        ulong[] GetULongArray(ulong[] x);
+
+        /** */
+        Guid GetGuid(Guid x);
+
+        /** */
+        Guid[] GetGuidArray(Guid[] x);
+
+        /** */
+        DateTime GetDateTime(DateTime x);
+
+        /** */
+        DateTime[] GetDateTimeArray(DateTime[] x);
+
+        /** */
+        List<DateTime> GetDateTimeList(ICollection<DateTime> x);
+
+        /** */
+        TimeSpan GetTimeSpan(TimeSpan x);
+
+        /** */
+        TimeSpan[] GetTimeSpanArray(TimeSpan[] x);
+
+        /** */
+        bool GetBool(bool x);
+
+        /** */
+        bool[] GetBoolArray(bool[] x);
+
+        /** */
+        float GetFloat(float x);
+
+        /** */
+        float[] GetFloatArray(float[] x);
+
+        /** */
+        double GetDouble(double x);
+
+        /** */
+        double[] GetDoubleArray(double[] x);
+
+        /** */
+        decimal GetDecimal(decimal x);
+
+        /** */
+        decimal[] GetDecimalArray(decimal[] x);
+
+        /** */
+        string GetString(string x);
+
+        /** */
+        string[] GetStringArray(string[] x);
+    }
+}
diff --git a/modules/platforms/dotnet/Apache.Ignite.BenchmarkDotNet/Program.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Services/ITestServiceGenericMethods.cs
similarity index 68%
copy from modules/platforms/dotnet/Apache.Ignite.BenchmarkDotNet/Program.cs
copy to modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Services/ITestServiceGenericMethods.cs
index 624f720..2000e47 100644
--- a/modules/platforms/dotnet/Apache.Ignite.BenchmarkDotNet/Program.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Services/ITestServiceGenericMethods.cs
@@ -1,4 +1,4 @@
-/*
+/*
  * 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.
@@ -15,22 +15,14 @@
  * limitations under the License.
  */
 
-namespace Apache.Ignite.BenchmarkDotNet
+namespace Apache.Ignite.Core.Tests.Client.Services
 {
-    using Apache.Ignite.BenchmarkDotNet.ThinClient;
-    using global::BenchmarkDotNet.Running;
-
     /// <summary>
-    /// Benchmark runner.
+    /// Test service interface with generic methods.
     /// </summary>
-    public static class Program
+    public interface ITestServiceGenericMethods
     {
-        /// <summary>
-        /// Main.
-        /// </summary>
-        public static void Main()
-        {
-            BenchmarkRunner.Run<ThinClientCachePutBenchmark>();
-        }
+        /** */
+        T GetGeneric<T>(T x);
     }
-}
+}
\ No newline at end of file
diff --git a/modules/platforms/dotnet/Apache.Ignite.BenchmarkDotNet/Program.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Services/ITestServiceOverloads.cs
similarity index 55%
copy from modules/platforms/dotnet/Apache.Ignite.BenchmarkDotNet/Program.cs
copy to modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Services/ITestServiceOverloads.cs
index 624f720..a0a9d2e 100644
--- a/modules/platforms/dotnet/Apache.Ignite.BenchmarkDotNet/Program.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Services/ITestServiceOverloads.cs
@@ -1,4 +1,4 @@
-/*
+/*
  * 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.
@@ -15,22 +15,43 @@
  * limitations under the License.
  */
 
-namespace Apache.Ignite.BenchmarkDotNet
+namespace Apache.Ignite.Core.Tests.Client.Services
 {
-    using Apache.Ignite.BenchmarkDotNet.ThinClient;
-    using global::BenchmarkDotNet.Running;
+    using Apache.Ignite.Core.Tests.Client.Cache;
 
     /// <summary>
-    /// Benchmark runner.
+    /// Test interface for overload resolution.
     /// </summary>
-    public static class Program
+    public interface ITestServiceOverloads
     {
-        /// <summary>
-        /// Main.
-        /// </summary>
-        public static void Main()
-        {
-            BenchmarkRunner.Run<ThinClientCachePutBenchmark>();
-        }
+        /** */
+        bool Foo();
+
+        /** */
+        int Foo(int x);
+        
+        /** */
+        int Foo(uint x);
+        
+        /** */
+        int Foo(byte x);
+        
+        /** */
+        int Foo(short x);
+        
+        /** */
+        int Foo(ushort x);
+        
+        /** */
+        int Foo(Person x);
+        
+        /** */
+        int Foo(int[] x);
+        
+        /** */
+        int Foo(object[] x);
+        
+        /** */
+        int Foo(Person[] x);
     }
-}
+}
\ No newline at end of file
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Services/ServicesClientTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Services/ServicesClientTest.cs
new file mode 100644
index 0000000..f74b0b4
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Services/ServicesClientTest.cs
@@ -0,0 +1,588 @@
+/*
+ * 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.Client.Services
+{
+    using System;
+    using System.Collections.Generic;
+    using System.Linq;
+    using Apache.Ignite.Core.Binary;
+    using Apache.Ignite.Core.Client;
+    using Apache.Ignite.Core.Client.Services;
+    using Apache.Ignite.Core.Services;
+    using Apache.Ignite.Core.Tests.Client.Cache;
+    using Apache.Ignite.Core.Tests.Services;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests for <see cref="IServicesClient"/>.
+    /// </summary>
+    public class ServicesClientTest : ClientTestBase
+    {
+        /** */
+        private const string ServiceName = "SVC_NAME";
+
+        /// <summary>
+        /// Initializes a new instance of <see cref="ServicesClientTest"/> class.
+        /// </summary>
+        public ServicesClientTest() : base(2)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Tears down the test.
+        /// </summary>
+        [TearDown]
+        public void TestTearDown()
+        {
+            ServerServices.CancelAll();
+
+            TestUtils.AssertHandleRegistryIsEmpty(1000, Ignition.GetAll().ToArray());
+        }
+
+        /// <summary>
+        /// Tests the basic success path.
+        /// </summary>
+        [Test]
+        public void TestBasicServiceCall()
+        {
+            var svc = DeployAndGetTestService();
+
+            var res = svc.IntMethod();
+
+            Assert.AreEqual(42, res);
+        }
+
+        /// <summary>
+        /// Tests that void method can be called and the lack of result is handled correctly.
+        ///
+        /// - Invoke void method
+        /// - Verify invoke count on server
+        /// </summary>
+        [Test]
+        public void TestVoidMethodCall()
+        {
+            var svc = DeployAndGetTestService();
+            var expectedCallCount = TestService.CallCount + 1;
+
+            svc.VoidMethod();
+
+            Assert.AreEqual(expectedCallCount, TestService.CallCount);
+        }
+
+        /// <summary>
+        /// Tests that objects can be passed to and from service methods.
+        /// </summary>
+        [Test]
+        public void TestObjectMethodCall()
+        {
+            var svc = DeployAndGetTestService();
+
+            var res = svc.PersonMethod(new Person(1));
+
+            Assert.AreEqual(2, res.Id);
+        }
+
+        /// <summary>
+        /// Tests that generic method can be called on a service.
+        /// </summary>
+        [Test]
+        [Ignore("IGNITE-13351")]
+        public void TestGenericMethodCall()
+        {
+            var svcName = TestUtils.TestName;
+            ServerServices.DeployClusterSingleton(svcName, new TestServiceGenericMethods());
+
+            var svc = Client.GetServices().GetServiceProxy<ITestServiceGenericMethods>(svcName);
+
+            Assert.AreEqual("1", svc.GetGeneric("1"));
+        }
+
+        /// <summary>
+        /// Tests that <see cref="IServicesClient.WithKeepBinary"/> call causes service invocation results to be
+        /// returned in serialized form.
+        ///
+        /// - Enable binary mode
+        /// - Verify that service invocation result is <see cref="IBinaryObject"/>
+        /// </summary>
+        [Test]
+        public void TestClientKeepBinaryReturnsServiceInvocationResultInBinaryMode()
+        {
+            var svc = DeployAndGetTestService<ITestServiceClient>(s => s.WithKeepBinary());
+
+            var person = Client.GetBinary().ToBinary<IBinaryObject>(new Person(5));
+            var res = svc.PersonMethod(person);
+
+            Assert.AreEqual(6, res.GetField<int>("Id"));
+        }
+
+        /// <summary>
+        /// Tests that <see cref="IServicesClient.WithServerKeepBinary"/> call causes service invocation arguments
+        /// to be passed to the service (on server node) in serialized form.
+        ///
+        /// - Enable server-side binary mode
+        /// - Check that server-side service receives <see cref="IBinaryObject"/>, modifies it and returns back
+        /// </summary>
+        [Test]
+        public void TestServerKeepBinaryPassesServerSideArgumentsInBinaryMode()
+        {
+            var svc = DeployAndGetTestService<ITestServiceClient>(s => s.WithServerKeepBinary());
+
+            var res = svc.PersonMethodBinary(new Person(1));
+
+            Assert.AreEqual(2, res.Id);
+        }
+
+        /// <summary>
+        /// Tests that <see cref="IServicesClient.WithServerKeepBinary"/> combined with
+        /// <see cref="IServicesClient.WithKeepBinary"/> uses binary objects both on client and server sides.
+        ///
+        /// - Enable server and client binary mode
+        /// - Check that both server and client operate on <see cref="IBinaryObject"/> instances
+        /// </summary>
+        [Test]
+        public void TestServerAndClientKeepBinaryPassesBinaryObjectsOnServerAndClient()
+        {
+            var svc = DeployAndGetTestService(s => s.WithKeepBinary().WithServerKeepBinary());
+
+            var person = Client.GetBinary().ToBinary<IBinaryObject>(new Person(-2));
+            var res = svc.PersonMethodBinary(person);
+
+            Assert.AreEqual(-1, res.GetField<int>("Id"));
+        }
+
+        /// <summary>
+        /// Tests that property getters/setters can be invoked on a remote service.
+        /// </summary>
+        [Test]
+        public void TestPropertyCalls()
+        {
+            var svc = DeployAndGetTestService();
+
+            // Primitive.
+            svc.IntProperty = 99;
+            Assert.AreEqual(99, svc.IntProperty);
+
+            // Object.
+            svc.PersonProperty= new Person(123);
+            Assert.AreEqual(123, svc.PersonProperty.Id);
+        }
+
+        /// <summary>
+        /// Tests that object array can be passed to and from the remote service.
+        /// </summary>
+        [Test]
+        public void TestObjectArray()
+        {
+            var svc = DeployAndGetTestService();
+
+            var res = svc.PersonArrayMethod(new[] {new Person(10), new Person(20)});
+
+            Assert.AreEqual(new[] {12, 22}, res.Select(p => p.Id));
+        }
+
+        /// <summary>
+        /// Tests that object array can be passed to and from the remote service in binary mode.
+        /// </summary>
+        [Test]
+        public void TestObjectArrayBinary()
+        {
+            var svc = DeployAndGetTestService(s => s.WithKeepBinary().WithServerKeepBinary());
+
+            var persons = new[] {new Person(10), new Person(20)}
+                .Select(p => Client.GetBinary().ToBinary<IBinaryObject>(p))
+                .ToArray();
+
+            var res = svc.PersonArrayMethodBinary(persons);
+
+            Assert.AreEqual(new[] {12, 22}, res.Select(p => p.GetField<int>("Id")));
+        }
+
+        /// <summary>
+        /// Tests all primitive and built-in types used as parameters and return values.
+        /// </summary>
+        [Test]
+        public void TestAllArgumentTypes()
+        {
+            ServerServices.DeployClusterSingleton(ServiceName, new TestServiceDataTypes());
+            var svc = Client.GetServices().GetServiceProxy<ITestServiceDataTypes>(ServiceName);
+
+            Assert.AreEqual(2, svc.GetByte(1));
+            Assert.AreEqual(new byte[] {3, 4, 5}, svc.GetByteArray(new byte[] {2, 3, 4}));
+
+            Assert.AreEqual(3, svc.GetSbyte(2));
+            Assert.AreEqual(new sbyte[] {-4, 6}, svc.GetSbyteArray(new sbyte[] {-5, 5}));
+
+            Assert.AreEqual(3, svc.GetShort(2));
+            Assert.AreEqual(new short[] {-4, 6}, svc.GetShortArray(new short[] {-5, 5}));
+
+            Assert.AreEqual(3, svc.GetUShort(2));
+            Assert.AreEqual(new ushort[] {1, 6}, svc.GetUShortArray(new ushort[] {0, 5}));
+
+            Assert.AreEqual(3, svc.GetInt(2));
+            Assert.AreEqual(new [] {-4, 6}, svc.GetIntArray(new[] {-5, 5}));
+
+            Assert.AreEqual(3, svc.GetUInt(2));
+            Assert.AreEqual(new uint[] {1, 6}, svc.GetUIntArray(new uint[] {0, 5}));
+
+            Assert.AreEqual(long.MaxValue - 9, svc.GetLong(long.MaxValue - 10));
+            Assert.AreEqual(new [] {long.MinValue + 1, 6}, svc.GetLongArray(new[] {long.MinValue, 5}));
+
+            Assert.AreEqual(ulong.MaxValue - 9, svc.GetULong(ulong.MaxValue - 10));
+            Assert.AreEqual(new ulong[] {1, 10}, svc.GetULongArray(new ulong[] {0, 9}));
+
+            Assert.AreEqual('d', svc.GetChar('c'));
+            Assert.AreEqual(new[] {'b', 'c'}, svc.GetCharArray(new[]{'a', 'b'}));
+
+            var guid = Guid.NewGuid();
+            Assert.AreEqual(guid, svc.GetGuid(guid));
+            Assert.AreEqual(new[] {guid, Guid.Empty}, svc.GetGuidArray(new[] {guid, Guid.Empty}));
+
+            var dt = DateTime.Now;
+            Assert.AreEqual(dt.AddDays(1), svc.GetDateTime(dt));
+            Assert.AreEqual(new[] {dt.AddDays(1), dt.AddDays(2)}, svc.GetDateTimeArray(new[] {dt, dt.AddDays(1)}));
+            Assert.AreEqual(new List<DateTime> {dt.AddDays(1), dt.AddDays(2)},
+                svc.GetDateTimeList(new[] {dt, dt.AddDays(1)}.ToList()));
+
+            var ts = TimeSpan.FromSeconds(25);
+            var minuteTs = TimeSpan.FromMinutes(1);
+            Assert.AreEqual(ts.Add(minuteTs), svc.GetTimeSpan(ts));
+            Assert.AreEqual(new[] {ts.Add(minuteTs), minuteTs}, svc.GetTimeSpanArray(new[] {ts, TimeSpan.Zero}));
+
+            Assert.AreEqual(true, svc.GetBool(false));
+            Assert.AreEqual(new[] {true, false}, svc.GetBoolArray(new[] {false, true}));
+
+            Assert.AreEqual(1.5f, svc.GetFloat(0.5f));
+            Assert.AreEqual(new[] {-0.5f, 1.1f}, svc.GetFloatArray(new[] {-1.5f, 0.1f}));
+
+            Assert.AreEqual(1.5d, svc.GetDouble(0.5d));
+            Assert.AreEqual(new[] {-7.02d, 1.1d}, svc.GetDoubleArray(new[] {-8.02d, 0.1d}));
+
+            Assert.AreEqual(1.666m, svc.GetDecimal(0.666m));
+            Assert.AreEqual(new[] {-7.66m, 1.33m}, svc.GetDecimalArray(new[] {-8.66m, 0.33m}));
+
+            Assert.AreEqual("foo", svc.GetString("FOO"));
+            Assert.AreEqual(new[]{"foo", "bar"}, svc.GetStringArray(new[]{"FoO", "bAr"}));
+        }
+
+        /// <summary>
+        /// Tests that overloaded methods are resolved correctly.
+        ///
+        /// - Invoke multiple overloads of the same method
+        /// - Check that correct overload is invoked based on the return value
+        /// </summary>
+        [Test]
+        public void TestOverloadResolution()
+        {
+            var svcName = TestUtils.TestName;
+            ServerServices.DeployClusterSingleton(svcName, new TestServiceOverloads());
+            var svc = Client.GetServices().GetServiceProxy<ITestServiceOverloads>(svcName);
+
+            Assert.AreEqual(true, svc.Foo());
+            Assert.AreEqual(1, svc.Foo(default(int)));
+            Assert.AreEqual(3, svc.Foo(default(byte)));
+            Assert.AreEqual(4, svc.Foo(default(short)));
+            Assert.AreEqual(6, svc.Foo(new Person()));
+            Assert.AreEqual(8, svc.Foo(new[] {1}));
+            Assert.AreEqual(9, svc.Foo(new[] {new object()}));
+
+            // Unsigned types are not preserved by the binary protocol and resolve to signed counterparts.
+            Assert.AreEqual(1, svc.Foo(default(uint)));
+            Assert.AreEqual(4, svc.Foo(default(ushort)));
+
+            // Array types are not distinguished.
+            Assert.AreEqual(9, svc.Foo(new[] {new Person(0)}));
+        }
+
+        /// <summary>
+        /// Tests that thin client can call Java services.
+        /// </summary>
+        [Test]
+        public void TestJavaServiceCall()
+        {
+            var serviceName = TestUtils.DeployJavaService(Ignition.GetIgnite());
+            var svc = Client.GetServices().GetServiceProxy<IJavaService>(serviceName);
+            var binSvc = Client.GetServices()
+                .WithKeepBinary()
+                .WithServerKeepBinary()
+                .GetServiceProxy<IJavaService>(serviceName);
+
+            Assert.IsTrue(svc.isInitialized());
+            Assert.IsTrue(svc.isExecuted());
+            Assert.IsFalse(svc.isCancelled());
+
+            // Primitives.
+            Assert.AreEqual(4, svc.test((byte) 3));
+            Assert.AreEqual(5, svc.test((short) 4));
+            Assert.AreEqual(6, svc.test(5));
+            Assert.AreEqual(6, svc.test((long) 5));
+            Assert.AreEqual(3.8f, svc.test(2.3f));
+            Assert.AreEqual(5.8, svc.test(3.3));
+            Assert.IsFalse(svc.test(true));
+            Assert.AreEqual('b', svc.test('a'));
+            Assert.AreEqual("Foo!", svc.test("Foo"));
+
+            // Nullables (Java wrapper types).
+            Assert.AreEqual(4, svc.testWrapper(3));
+            Assert.AreEqual(5, svc.testWrapper((short?) 4));
+            Assert.AreEqual(6, svc.testWrapper((int?)5));
+            Assert.AreEqual(6, svc.testWrapper((long?) 5));
+            Assert.AreEqual(3.8f, svc.testWrapper(2.3f));
+            Assert.AreEqual(5.8, svc.testWrapper(3.3));
+            Assert.AreEqual(false, svc.testWrapper(true));
+            Assert.AreEqual('b', svc.testWrapper('a'));
+
+            // Arrays.
+            Assert.AreEqual(new byte[] {2, 3, 4}, svc.testArray(new byte[] {1, 2, 3}));
+            Assert.AreEqual(new short[] {2, 3, 4}, svc.testArray(new short[] {1, 2, 3}));
+            Assert.AreEqual(new[] {2, 3, 4}, svc.testArray(new[] {1, 2, 3}));
+            Assert.AreEqual(new long[] {2, 3, 4}, svc.testArray(new long[] {1, 2, 3}));
+            Assert.AreEqual(new float[] {2, 3, 4}, svc.testArray(new float[] {1, 2, 3}));
+            Assert.AreEqual(new double[] {2, 3, 4}, svc.testArray(new double[] {1, 2, 3}));
+            Assert.AreEqual(new[] {"a1", "b1"}, svc.testArray(new [] {"a", "b"}));
+            Assert.AreEqual(new[] {'c', 'd'}, svc.testArray(new[] {'b', 'c'}));
+            Assert.AreEqual(new[] {false, true, false}, svc.testArray(new[] {true, false, true}));
+
+            // Nulls.
+            Assert.AreEqual(9, svc.testNull(8));
+            Assert.IsNull(svc.testNull(null));
+
+            // params / varargs.
+            Assert.AreEqual(5, svc.testParams(1, 2, 3, 4, "5"));
+            Assert.AreEqual(0, svc.testParams());
+
+            // Overloads.
+            Assert.AreEqual(3, svc.test(2, "1"));
+            Assert.AreEqual(3, svc.test("1", 2));
+
+            // Dates & Timestamps: not supported in Thin Client Services.
+            var ex = Assert.Throws<IgniteClientException>(() => svc.test(DateTime.UtcNow));
+            StringAssert.StartsWith("Failed to resolve .NET class 'System.DateTime' in Java", ex.Message);
+
+            // Guid.
+            var guid = Guid.NewGuid();
+
+            Assert.AreEqual(guid, svc.test(guid));
+            Assert.AreEqual(guid, svc.testNullUUID(guid));
+            Assert.IsNull(svc.testNullUUID(null));
+            Assert.AreEqual(guid, svc.testArray(new Guid?[] {guid})[0]);
+
+            // Binary object.
+            Assert.AreEqual(15,
+                binSvc.testBinaryObject(
+                    Client.GetBinary().ToBinary<IBinaryObject>(new ServicesTest.PlatformComputeBinarizable {Field = 6}))
+                    .GetField<int>("Field"));
+
+            // Binary object array.
+            var arr  = new[] {10, 11, 12}.Select(
+                x => new ServicesTest.PlatformComputeBinarizable {Field = x}).ToArray();
+
+            var binArr = arr.Select(Client.GetBinary().ToBinary<IBinaryObject>).ToArray();
+
+            Assert.AreEqual(new[] {11, 12, 13}, binSvc.testBinaryObjectArray(binArr)
+                .Select(x => x.GetField<int>("Field")));
+        }
+
+        /// <summary>
+        /// Tests that specifying custom cluster group causes service calls to be routed to selected servers.
+        ///
+        /// - Deploy the service on every server node with DeployNodeSingleton
+        /// - For every server node:
+        /// - Get a cluster group of a single node
+        /// - Invoke service method that returns local node id
+        /// - Check that specified server has been used to invoke the service
+        /// </summary>
+        [Test]
+        public void TestServicesWithCustomClusterGroupInvokeOnSpecifiedNodes()
+        {
+            ServerServices.DeployNodeSingleton(ServiceName, new TestService());
+
+            foreach (var ignite in Ignition.GetAll())
+            {
+                var node = ignite.GetCluster().GetLocalNode();
+                var clusterGroup = Client.GetCluster().ForPredicate(n => n.Id == node.Id);
+                var svc = clusterGroup.GetServices();
+
+                Assert.AreSame(clusterGroup, svc.ClusterGroup);
+                Assert.AreEqual(node.Id, clusterGroup.GetNodes().Single().Id);
+
+                var actualNodeId = svc.GetServiceProxy<ITestService>(ServiceName).GetNodeId();
+                Assert.AreEqual(node.Id, actualNodeId);
+            }
+        }
+
+        /// <summary>
+        /// Tests that empty cluster group causes exception on service call.
+        ///
+        /// - Create an empty cluster group
+        /// - Get services over that group
+        /// - Execute service method, verify exception
+        /// </summary>
+        [Test]
+        public void TestEmptyClusterGroupThrowsError()
+        {
+            ServerServices.DeployNodeSingleton(ServiceName, new TestService());
+
+            var clusterGroup = Client.GetCluster().ForPredicate(_ => false);
+            var svc = clusterGroup.GetServices().GetServiceProxy<ITestService>(ServiceName);
+
+            var ex = Assert.Throws<IgniteClientException>(() => svc.VoidMethod());
+            Assert.AreEqual("Cluster group is empty", ex.Message);
+        }
+
+        /// <summary>
+        /// Tests that a custom cluster group that does not have any server nodes with specified service produces
+        /// a correct exception.
+        ///
+        /// - Deploy the service to node X
+        /// - Create client cluster group with node Y
+        /// - Call service, verify exception
+        /// </summary>
+        [Test]
+        public void TestClusterGroupWithoutMatchingServiceNodesThrowsError()
+        {
+            var ignite = Ignition.GetIgnite();
+            var node = ignite.GetCluster().GetLocalNode();
+
+            ignite.GetCluster()
+                .ForNodes(node)
+                .GetServices()
+                .DeployClusterSingleton(ServiceName, new TestService());
+
+            var svc = Client.GetCluster()
+                .ForPredicate(n => n.Id != node.Id)
+                .GetServices()
+                .GetServiceProxy<ITestService>(ServiceName);
+
+            var ex = Assert.Throws<IgniteClientException>(() => svc.VoidMethod());
+            Assert.AreEqual("Failed to find deployed service: " + ServiceName, ex.Message);
+        }
+
+        /// <summary>
+        /// Tests that lingering service calls cause timeout exception when WithTimeout is used.
+        ///
+        /// - Deploy the service
+        /// - Get a service proxy with a timeout
+        /// - Execute a method that takes a long time, verify that timeout setting takes effect
+        /// </summary>
+        [Test]
+        [Ignore("IGNITE-13360")]
+        public void TestTimeout()
+        {
+            var svc = DeployAndGetTestService();
+
+            var ex = Assert.Throws<IgniteClientException>(() => svc.Sleep(TimeSpan.FromSeconds(3)));
+
+            Assert.AreEqual("timed out", ex.Message);
+        }
+
+        /// <summary>
+        /// Tests that lingering service calls cause timeout exception when WithTimeout is used.
+        ///
+        /// - Deploy the service
+        /// - Get a service proxy with a timeout
+        /// - Execute a method that takes a long time, verify that timeout setting takes effect
+        /// </summary>
+        [Test]
+        [Ignore("IGNITE-13360")]
+        public void TestJavaServiceTimeout()
+        {
+            var svcName = TestUtils.DeployJavaService(Ignition.GetIgnite());
+
+            var svc = Client.GetServices()
+                .GetServiceProxy<IJavaService>(svcName);
+
+            var ex = Assert.Throws<IgniteClientException>(() => svc.sleep(2000));
+
+            Assert.AreEqual("timed out", ex.Message);
+        }
+
+        /// <summary>
+        /// Tests that exception in service is propagated to the client and service is still operational.
+        /// </summary>
+        [Test]
+        public void TestExceptionInServiceIsPropagatedToClient()
+        {
+            var svc = DeployAndGetTestService();
+
+            var ex = Assert.Throws<IgniteClientException>(() => svc.ExceptionalMethod());
+
+            Assert.AreEqual("Failed to invoke platform service, see server logs for details", ex.Message);
+        }
+
+        /// <summary>
+        /// Tests that invoking a service that does not exist causes a correct exception.
+        /// </summary>
+        [Test]
+        public void TestNonExistentServiceNameCausesClientException()
+        {
+            var svc = Client.GetServices().GetServiceProxy<ITestService>(ServiceName);
+
+            var ex = Assert.Throws<IgniteClientException>(() => svc.VoidMethod());
+            Assert.AreEqual(ClientStatusCode.Fail, ex.StatusCode);
+        }
+
+        /// <summary>
+        /// Tests async method calls.
+        /// </summary>
+        [Test]
+        [Ignore("IGNITE-13343")]
+        public void TestAsyncServiceCalls()
+        {
+            var svc = DeployAndGetTestService();
+
+            var task = svc.AsyncMethod();
+            task.Wait();
+
+            Assert.AreEqual(1, task.Result);
+        }
+
+        /// <summary>
+        /// Deploys test service and returns client-side proxy.
+        /// </summary>
+        private ITestService DeployAndGetTestService(Func<IServicesClient, IServicesClient> transform = null)
+        {
+            return DeployAndGetTestService<ITestService>(transform);
+        }
+
+        /// <summary>
+        /// Deploys test service and returns client-side proxy.
+        /// </summary>
+        private T DeployAndGetTestService<T>(Func<IServicesClient, IServicesClient> transform = null) where T : class
+        {
+            ServerServices.DeployClusterSingleton(ServiceName, new TestService());
+
+            var services = Client.GetServices();
+
+            if (transform != null)
+            {
+                services = transform(services);
+            }
+
+            return services.GetServiceProxy<T>(ServiceName);
+        }
+
+        /// <summary>
+        /// Gets server-side Services.
+        /// </summary>
+        private static IServices ServerServices
+        {
+            get { return Ignition.GetIgnite().GetServices(); }
+        }
+    }
+}
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Services/TestService.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Services/TestService.cs
new file mode 100644
index 0000000..c61b43a
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Services/TestService.cs
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.Core.Tests.Client.Services
+{
+    using System;
+    using System.Linq;
+    using System.Threading;
+    using System.Threading.Tasks;
+    using Apache.Ignite.Core.Binary;
+    using Apache.Ignite.Core.Resource;
+    using Apache.Ignite.Core.Services;
+    using Apache.Ignite.Core.Tests.Client.Cache;
+
+    /// <summary>
+    /// Test service.
+    /// </summary>
+    public class TestService : ITestService, IService
+    {
+        /** */
+        [InstanceResource]
+        private readonly IIgnite _ignite = null;
+
+        /** */
+        public const string ExceptionText = "Some error";
+
+        /** */
+        public static int CallCount { get; set; }
+
+        /** <inheritdoc /> */
+        public int IntProperty { get; set; }
+
+        /** <inheritdoc /> */
+        public Person PersonProperty { get; set; }
+
+        /** <inheritdoc /> */
+        public void VoidMethod()
+        {
+            CallCount++;
+        }
+
+        /** <inheritdoc /> */
+        public int IntMethod()
+        {
+            return 42;
+        }
+
+        /** <inheritdoc /> */
+        public void ExceptionalMethod()
+        {
+            throw new ArithmeticException(ExceptionText);
+        }
+
+        /** <inheritdoc /> */
+        public Task<int> AsyncMethod()
+        {
+            var tcs = new TaskCompletionSource<int>();
+            new Timer(_ => tcs.SetResult(1)).Change(500, -1);
+            return tcs.Task;        }
+
+        /** <inheritdoc /> */
+        public Person PersonMethod(Person person)
+        {
+            return new Person(person.Id + 1);
+        }
+
+        /** <inheritdoc /> */
+        public IBinaryObject PersonMethodBinary(IBinaryObject person)
+        {
+            return person
+                .ToBuilder()
+                .SetField("Id", person.GetField<int>("Id") + 1)
+                .Build();
+        }
+
+        /** <inheritdoc /> */
+        public Person[] PersonArrayMethod(Person[] persons)
+        {
+            return persons.Select(p => new Person(p.Id + 2)).ToArray();
+        }
+
+        /** <inheritdoc /> */
+        public IBinaryObject[] PersonArrayMethodBinary(IBinaryObject[] persons)
+        {
+            return persons
+                .Select(p => p.ToBuilder().SetIntField("Id", p.GetField<int>("Id") + 2).Build())
+                .ToArray();
+        }
+
+        /** <inheritdoc /> */
+        public void Sleep(TimeSpan delay)
+        {
+            Thread.Sleep(delay);
+        }
+
+        /** <inheritdoc /> */
+        public Guid GetNodeId()
+        {
+            return _ignite.GetCluster().GetLocalNode().Id;
+        }
+
+        /** <inheritdoc /> */
+        public void Init(IServiceContext context)
+        {
+            // No-op.
+        }
+
+        /** <inheritdoc /> */
+        public void Execute(IServiceContext context)
+        {
+            // No-op.
+        }
+
+        /** <inheritdoc /> */
+        public void Cancel(IServiceContext context)
+        {
+            // No-op.
+        }
+    }
+}
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Services/TestServiceDataTypes.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Services/TestServiceDataTypes.cs
new file mode 100644
index 0000000..ae5cde9
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Services/TestServiceDataTypes.cs
@@ -0,0 +1,258 @@
+/*
+ * 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.Client.Services
+{
+    using System;
+    using System.Collections.Generic;
+    using System.Linq;
+    using Apache.Ignite.Core.Services;
+
+    /// <summary>
+    /// Service for testing various data types passing.
+    /// </summary>
+    public class TestServiceDataTypes : ITestServiceDataTypes, IService
+    {
+        /** <inheritdoc /> */
+        public byte GetByte(byte x)
+        {
+            return ++x;
+        }
+
+        /** <inheritdoc /> */
+        public byte[] GetByteArray(byte[] x)
+        {
+            return x.Select(b => ++b).ToArray();
+        }
+
+        /** <inheritdoc /> */
+        public sbyte GetSbyte(sbyte x)
+        {
+            return ++x;
+        }
+
+        /** <inheritdoc /> */
+        public sbyte[] GetSbyteArray(sbyte[] x)
+        {
+            return x.Select(b => ++b).ToArray();
+        }
+
+        /** <inheritdoc /> */
+        public char GetChar(char x)
+        {
+            return ++x;
+        }
+
+        /** <inheritdoc /> */
+        public char[] GetCharArray(char[] x)
+        {
+            return x.Select(b => ++b).ToArray();
+        }
+
+        /** <inheritdoc /> */
+        public short GetShort(short x)
+        {
+            return ++x;
+        }
+
+        /** <inheritdoc /> */
+        public short[] GetShortArray(short[] x)
+        {
+            return x.Select(s => ++s).ToArray();
+        }
+
+        /** <inheritdoc /> */
+        public ushort GetUShort(ushort x)
+        {
+            return ++x;
+        }
+
+        /** <inheritdoc /> */
+        public ushort[] GetUShortArray(ushort[] x)
+        {
+            return x.Select(s => ++s).ToArray();
+        }
+
+        /** <inheritdoc /> */
+        public int GetInt(int x)
+        {
+            return ++x;
+        }
+
+        /** <inheritdoc /> */
+        public int[] GetIntArray(int[] x)
+        {
+            return x.Select(s => ++s).ToArray();
+        }
+
+        /** <inheritdoc /> */
+        public uint GetUInt(uint x)
+        {
+            return ++x;
+        }
+
+        /** <inheritdoc /> */
+        public uint[] GetUIntArray(uint[] x)
+        {
+            return x.Select(s => ++s).ToArray();
+        }
+
+        /** <inheritdoc /> */
+        public long GetLong(long x)
+        {
+            return ++x;
+        }
+
+        /** <inheritdoc /> */
+        public long[] GetLongArray(long[] x)
+        {
+            return x.Select(s => ++s).ToArray();
+        }
+
+        /** <inheritdoc /> */
+        public ulong GetULong(ulong x)
+        {
+            return ++x;
+        }
+
+        /** <inheritdoc /> */
+        public ulong[] GetULongArray(ulong[] x)
+        {
+            return x.Select(b => ++b).ToArray();
+        }
+
+        /** <inheritdoc /> */
+        public Guid GetGuid(Guid x)
+        {
+            return x;
+        }
+
+        /** <inheritdoc /> */
+        public Guid[] GetGuidArray(Guid[] x)
+        {
+            return x.ToArray();
+        }
+
+        /** <inheritdoc /> */
+        public DateTime GetDateTime(DateTime x)
+        {
+            return x.AddDays(1);
+        }
+
+        /** <inheritdoc /> */
+        public DateTime[] GetDateTimeArray(DateTime[] x)
+        {
+            return x.Select(d => d.AddDays(1)).ToArray();
+        }
+
+        /** <inheritdoc /> */
+        public List<DateTime> GetDateTimeList(ICollection<DateTime> x)
+        {
+            return x.Select(d => d.AddDays(1)).ToList();
+        }
+
+        /** <inheritdoc /> */
+        public TimeSpan GetTimeSpan(TimeSpan x)
+        {
+            return x.Add(TimeSpan.FromMinutes(1));
+        }
+
+        /** <inheritdoc /> */
+        public TimeSpan[] GetTimeSpanArray(TimeSpan[] x)
+        {
+            return x.Select(b => b.Add(TimeSpan.FromMinutes(1))).ToArray();
+        }
+
+        /** <inheritdoc /> */
+        public bool GetBool(bool x)
+        {
+            return !x;
+        }
+
+        /** <inheritdoc /> */
+        public bool[] GetBoolArray(bool[] x)
+        {
+            return x.Select(b => !b).ToArray();
+        }
+
+        /** <inheritdoc /> */
+        public float GetFloat(float x)
+        {
+            return ++x;
+        }
+
+        /** <inheritdoc /> */
+        public float[] GetFloatArray(float[] x)
+        {
+            return x.Select(b => ++b).ToArray();
+        }
+
+        /** <inheritdoc /> */
+        public double GetDouble(double x)
+        {
+            return ++x;
+        }
+
+        /** <inheritdoc /> */
+        public double[] GetDoubleArray(double[] x)
+        {
+            return x.Select(b => ++b).ToArray();
+        }
+
+        /** <inheritdoc /> */
+        public decimal GetDecimal(decimal x)
+        {
+            return ++x;
+        }
+
+        /** <inheritdoc /> */
+        public decimal[] GetDecimalArray(decimal[] x)
+        {
+            return x.Select(b => ++b).ToArray();
+        }
+
+        /** <inheritdoc /> */
+        public string GetString(string x)
+        {
+            return x.ToLowerInvariant();
+        }
+
+        /** <inheritdoc /> */
+        public string[] GetStringArray(string[] x)
+        {
+            return x.Select(b => b.ToLowerInvariant()).ToArray();
+        }
+
+        /** <inheritdoc /> */
+        public void Init(IServiceContext context)
+        {
+            // No-op.
+        }
+
+        /** <inheritdoc /> */
+        public void Execute(IServiceContext context)
+        {
+            // No-op.
+        }
+
+        /** <inheritdoc /> */
+        public void Cancel(IServiceContext context)
+        {
+            // No-op.
+        }
+    }
+}
diff --git a/modules/platforms/dotnet/Apache.Ignite.BenchmarkDotNet/Program.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Services/TestServiceGenericMethods.cs
similarity index 56%
copy from modules/platforms/dotnet/Apache.Ignite.BenchmarkDotNet/Program.cs
copy to modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Services/TestServiceGenericMethods.cs
index 624f720..39464a1 100644
--- a/modules/platforms/dotnet/Apache.Ignite.BenchmarkDotNet/Program.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Services/TestServiceGenericMethods.cs
@@ -1,4 +1,4 @@
-/*
+/*
  * 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.
@@ -15,22 +15,37 @@
  * limitations under the License.
  */
 
-namespace Apache.Ignite.BenchmarkDotNet
+namespace Apache.Ignite.Core.Tests.Client.Services
 {
-    using Apache.Ignite.BenchmarkDotNet.ThinClient;
-    using global::BenchmarkDotNet.Running;
+    using Apache.Ignite.Core.Services;
 
     /// <summary>
-    /// Benchmark runner.
+    /// Test service with generic methods.
     /// </summary>
-    public static class Program
+    public class TestServiceGenericMethods : ITestServiceGenericMethods, IService
     {
-        /// <summary>
-        /// Main.
-        /// </summary>
-        public static void Main()
+        /** <inheritdoc /> */
+        public T GetGeneric<T>(T x)
         {
-            BenchmarkRunner.Run<ThinClientCachePutBenchmark>();
+            return x;
+        }
+
+        /** <inheritdoc /> */
+        public void Init(IServiceContext context)
+        {
+            // No-op.
+        }
+
+        /** <inheritdoc /> */
+        public void Execute(IServiceContext context)
+        {
+            // No-op.
+        }
+
+        /** <inheritdoc /> */
+        public void Cancel(IServiceContext context)
+        {
+            // No-op.
         }
     }
 }
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Services/TestServiceOverloads.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Services/TestServiceOverloads.cs
new file mode 100644
index 0000000..20ee21c
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Services/TestServiceOverloads.cs
@@ -0,0 +1,106 @@
+/*
+ * 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.Client.Services
+{
+    using Apache.Ignite.Core.Services;
+    using Apache.Ignite.Core.Tests.Client.Cache;
+
+    /// <summary>
+    /// Tests service with overloaded methods.
+    /// </summary>
+    public class TestServiceOverloads : ITestServiceOverloads, IService
+    {
+        /** <inheritdoc /> */
+        public bool Foo()
+        {
+            return true;
+        }
+
+        /** <inheritdoc /> */
+        public int Foo(int x)
+        {
+            return 1;
+        }
+
+        /** <inheritdoc /> */
+        public int Foo(uint x)
+        {
+            return 2;
+        }
+
+        /** <inheritdoc /> */
+        public int Foo(byte x)
+        {
+            return 3;
+        }
+
+        /** <inheritdoc /> */
+        public int Foo(short x)
+        {
+            return 4;
+        }
+
+        /** <inheritdoc /> */
+        public int Foo(ushort x)
+        {
+            return 5;
+        }
+
+        /** <inheritdoc /> */
+        public int Foo(Person x)
+        {
+            return 6;
+        }
+
+        /** <inheritdoc /> */
+        public int Foo(int[] x)
+        {
+            return 8;
+        }
+
+        /** <inheritdoc /> */
+        public int Foo(object[] x)
+        {
+            return 9;
+        }
+
+        /** <inheritdoc /> */
+        public int Foo(Person[] x)
+        {
+            return 10;
+        }
+
+        /** <inheritdoc /> */
+        public void Init(IServiceContext context)
+        {
+            // No-op.
+        }
+
+        /** <inheritdoc /> */
+        public void Execute(IServiceContext context)
+        {
+            // No-op.
+        }
+
+        /** <inheritdoc /> */
+        public void Cancel(IServiceContext context)
+        {
+            // No-op.
+        }
+    }
+}
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteStartStopTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteStartStopTest.cs
index 9fa0afb..0c002d6 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteStartStopTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteStartStopTest.cs
@@ -51,7 +51,7 @@ namespace Apache.Ignite.Core.Tests
 
             Assert.IsNotNull(grid);
 
-            Assert.AreEqual(1, grid.GetCluster().GetNodes().Count);
+            Assert.AreEqual(1, grid.GetCluster().GetNodes().Count, "Unexpected number of nodes in the cluster.");
         }
 
         /// <summary>
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ProjectFilesTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ProjectFilesTest.cs
index 02c2bd4..1a7625c 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ProjectFilesTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ProjectFilesTest.cs
@@ -39,7 +39,7 @@ namespace Apache.Ignite.Core.Tests
                 .GetFiles("*.csproj", SearchOption.AllDirectories)
                 .Where(x => !x.FullName.ToLower().Contains("dotnetcore") && !x.FullName.Contains("Benchmark"))
                 .ToArray();
-            
+
             Assert.GreaterOrEqual(projFiles.Length, 7);
             CheckFiles(projFiles, x => !x.Contains("ToolsVersion=\"4.0\""), "Invalid csproj files: ");
         }
@@ -50,7 +50,7 @@ namespace Apache.Ignite.Core.Tests
         [Test]
         public void TestCsprojReleaseDocs()
         {
-            CheckFiles(GetReleaseCsprojFiles(), x => !GetReleaseSection(x).Contains("DocumentationFile"), 
+            CheckFiles(GetReleaseCsprojFiles(), x => !GetReleaseSection(x).Contains("DocumentationFile"),
                 "Missing XML doc in release mode: ");
         }
 
@@ -60,7 +60,7 @@ namespace Apache.Ignite.Core.Tests
         [Test]
         public void TestCsprojBuildSettings()
         {
-            CheckFiles(GetReleaseCsprojFiles(), x => GetReleaseSection(x).Contains("DefineConstants"), 
+            CheckFiles(GetReleaseCsprojFiles(), x => GetReleaseSection(x).Contains("DefineConstants"),
                 "Invalid constants in release mode: ");
         }
 
@@ -70,7 +70,7 @@ namespace Apache.Ignite.Core.Tests
         [Test]
         public void TestCsprojPdbSettings()
         {
-            CheckFiles(GetReleaseCsprojFiles(), x => !GetReleaseSection(x).Contains("<DebugType>none</DebugType>"), 
+            CheckFiles(GetReleaseCsprojFiles(), x => !GetReleaseSection(x).Contains("<DebugType>none</DebugType>"),
                 "Invalid DebugType in release mode: ");
         }
 
@@ -80,10 +80,62 @@ namespace Apache.Ignite.Core.Tests
         [Test]
         public void TestCsprojOptimizeCode()
         {
-            CheckFiles(GetReleaseCsprojFiles(), x => !GetReleaseSection(x).Contains("<Optimize>true</Optimize>"), 
+            CheckFiles(GetReleaseCsprojFiles(), x => !GetReleaseSection(x).Contains("<Optimize>true</Optimize>"),
                 "Invalid optimize setting in release mode: ");
         }
 
+#if NETCOREAPP
+        /// <summary>
+        /// Tests that all .cs files are included in the project.
+        /// </summary>
+        [Test]
+        public void TestAllCsharpFilesAreIncludedInProject()
+        {
+            var projFiles = TestUtils.GetDotNetSourceDir().GetFiles("*.csproj", SearchOption.AllDirectories)
+                .Where(x =>
+                    !x.Name.Contains("DotNetCore") && !x.Name.Contains("Benchmark") && !x.Name.Contains("Examples"));
+
+            var excludedFiles = new[]
+            {
+                "IgnitionStartTest.cs",
+                "Common\\TestFixtureSetUp.cs",
+                "Common\\TestFixtureTearDown.cs",
+                "Client\\Cache\\CacheTestAsyncAwait.cs"
+            };
+
+            Assert.Multiple(() =>
+            {
+                foreach (var projFile in projFiles)
+                {
+                    Assert.IsNotNull(projFile.Directory);
+
+                    var projFileText = File.ReadAllText(projFile.FullName);
+                    var csFiles = projFile.Directory.GetFiles("*.cs", SearchOption.AllDirectories);
+
+                    foreach (var csFile in csFiles)
+                    {
+                        // Csproj uses the same path separator on all platforms.
+                        var csFileRelativePath = Path.GetRelativePath(projFile.Directory.FullName, csFile.FullName)
+                            .Replace(Path.DirectorySeparatorChar, '\\');
+
+                        if (csFileRelativePath.StartsWith("bin\\") ||
+                            csFileRelativePath.StartsWith("obj\\") ||
+                            csFileRelativePath.Contains("DotNetCore") ||
+                            excludedFiles.Contains(csFileRelativePath))
+                        {
+                            continue;
+                        }
+
+                        Assert.IsTrue(
+                            projFileText.Contains(csFileRelativePath),
+                            string.Format("Project file '{0}' should contain file '{1}'", projFile.Name,
+                                csFileRelativePath));
+                    }
+                }
+            });
+        }
+#endif
+
         /// <summary>
         /// Gets the csproj files that go to the release binary package.
         /// </summary>
@@ -102,7 +154,7 @@ namespace Apache.Ignite.Core.Tests
         /// </summary>
         private static string GetReleaseSection(string csproj)
         {
-            return Regex.Match(csproj, @"<PropertyGroup[^>]*Release\|AnyCPU(.*?)<\/PropertyGroup>", 
+            return Regex.Match(csproj, @"<PropertyGroup[^>]*Release\|AnyCPU(.*?)<\/PropertyGroup>",
                 RegexOptions.Singleline).Value;
         }
 
@@ -130,7 +182,7 @@ namespace Apache.Ignite.Core.Tests
             var allowedFiles = new[]
             {
                 "BinaryStringTest.cs",
-                "BinarySelfTest.cs", 
+                "BinarySelfTest.cs",
                 "CacheDmlQueriesTest.cs",
                 "CacheTest.cs"
             };
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/IJavaService.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/IJavaService.cs
new file mode 100644
index 0000000..ccceaf7
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/IJavaService.cs
@@ -0,0 +1,169 @@
+/*
+ * 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.Services
+{
+    using System;
+    using System.Collections;
+    using System.Diagnostics.CodeAnalysis;
+    using Apache.Ignite.Core.Binary;
+
+    /// <summary>
+    /// Java service proxy interface.
+    /// </summary>
+    [SuppressMessage("ReSharper", "InconsistentNaming")]
+    public interface IJavaService
+    {
+        /** */
+        bool isCancelled();
+
+        /** */
+        bool isInitialized();
+
+        /** */
+        bool isExecuted();
+
+        /** */
+        byte test(byte x);
+
+        /** */
+        short test(short x);
+
+        /** */
+        int test(int x);
+
+        /** */
+        long test(long x);
+
+        /** */
+        float test(float x);
+
+        /** */
+        double test(double x);
+
+        /** */
+        char test(char x);
+
+        /** */
+        string test(string x);
+
+        /** */
+        bool test(bool x);
+
+        /** */
+        DateTime test(DateTime x);
+
+        /** */
+        Guid test(Guid x);
+
+        /** */
+        byte? testWrapper(byte? x);
+
+        /** */
+        short? testWrapper(short? x);
+
+        /** */
+        int? testWrapper(int? x);
+
+        /** */
+        long? testWrapper(long? x);
+
+        /** */
+        float? testWrapper(float? x);
+
+        /** */
+        double? testWrapper(double? x);
+
+        /** */
+        char? testWrapper(char? x);
+
+        /** */
+        bool? testWrapper(bool? x);
+
+        /** */
+        byte[] testArray(byte[] x);
+
+        /** */
+        short[] testArray(short[] x);
+
+        /** */
+        int[] testArray(int[] x);
+
+        /** */
+        long[] testArray(long[] x);
+
+        /** */
+        float[] testArray(float[] x);
+
+        /** */
+        double[] testArray(double[] x);
+
+        /** */
+        char[] testArray(char[] x);
+
+        /** */
+        string[] testArray(string[] x);
+
+        /** */
+        bool[] testArray(bool[] x);
+
+        /** */
+        DateTime?[] testArray(DateTime?[] x);
+
+        /** */
+        Guid?[] testArray(Guid?[] x);
+
+        /** */
+        int test(int x, string y);
+
+        /** */
+        int test(string x, int y);
+
+        /** */
+        int? testNull(int? x);
+
+        /** */
+        DateTime? testNullTimestamp(DateTime? x);
+
+        /** */
+        Guid? testNullUUID(Guid? x);
+
+        /** */
+        int testParams(params object[] args);
+
+        /** */
+        ServicesTest.PlatformComputeBinarizable testBinarizable(ServicesTest.PlatformComputeBinarizable x);
+
+        /** */
+        object[] testBinarizableArrayOfObjects(object[] x);
+
+        /** */
+        IBinaryObject[] testBinaryObjectArray(IBinaryObject[] x);
+
+        /** */
+        ServicesTest.PlatformComputeBinarizable[] testBinarizableArray(ServicesTest.PlatformComputeBinarizable[] x);
+
+        /** */
+        ICollection testBinarizableCollection(ICollection x);
+
+        /** */
+        IBinaryObject testBinaryObject(IBinaryObject x);
+
+        /** */
+        void sleep(long delayMs);
+    }
+}
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTest.cs
index 1982e20..ee0bdea 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTest.cs
@@ -18,9 +18,7 @@
 namespace Apache.Ignite.Core.Tests.Services
 {
     using System;
-    using System.Collections;
     using System.Collections.Generic;
-    using System.Diagnostics.CodeAnalysis;
     using System.IO;
     using System.Linq;
     using System.Runtime.Serialization.Formatters.Binary;
@@ -852,10 +850,9 @@ namespace Apache.Ignite.Core.Tests.Services
         public void TestCallJavaService()
         {
             // Deploy Java service
-            const string javaSvcName = "javaService";
-            DeployJavaService(javaSvcName);
+            var javaSvcName = TestUtils.DeployJavaService(Grid1);
 
-            // Verify decriptor
+            // Verify descriptor
             var descriptor = Services.GetServiceDescriptors().Single(x => x.Name == javaSvcName);
             Assert.AreEqual(javaSvcName, descriptor.Name);
 
@@ -968,9 +965,8 @@ namespace Apache.Ignite.Core.Tests.Services
         [Test]
         public void TestCallJavaServiceDynamicProxy()
         {
-            const string javaSvcName = "javaService";
-            DeployJavaService(javaSvcName);
-
+            // Deploy Java service
+            var javaSvcName = TestUtils.DeployJavaService(Grid1);
             var svc = Grid1.GetServices().GetDynamicServiceProxy(javaSvcName, true);
 
             // Basics
@@ -1045,17 +1041,6 @@ namespace Apache.Ignite.Core.Tests.Services
         }
 
         /// <summary>
-        /// Deploys the java service.
-        /// </summary>
-        private void DeployJavaService(string javaSvcName)
-        {
-            Grid1.GetCompute()
-                .ExecuteJavaTask<object>("org.apache.ignite.platform.PlatformDeployServiceTask", javaSvcName);
-
-            TestUtils.WaitForCondition(() => Services.GetServiceDescriptors().Any(x => x.Name == javaSvcName), 1000);
-        }
-
-        /// <summary>
         /// Tests the footer setting.
         /// </summary>
         [Test]
@@ -1505,148 +1490,6 @@ namespace Apache.Ignite.Core.Tests.Services
         }
 
         /// <summary>
-        /// Java service proxy interface.
-        /// </summary>
-        [SuppressMessage("ReSharper", "InconsistentNaming")]
-        public interface IJavaService
-        {
-            /** */
-            bool isCancelled();
-
-            /** */
-            bool isInitialized();
-
-            /** */
-            bool isExecuted();
-
-            /** */
-            byte test(byte x);
-
-            /** */
-            short test(short x);
-
-            /** */
-            int test(int x);
-
-            /** */
-            long test(long x);
-
-            /** */
-            float test(float x);
-
-            /** */
-            double test(double x);
-
-            /** */
-            char test(char x);
-
-            /** */
-            string test(string x);
-
-            /** */
-            bool test(bool x);
-
-            /** */
-            DateTime test(DateTime x);
-
-            /** */
-            Guid test(Guid x);
-
-            /** */
-            byte? testWrapper(byte? x);
-
-            /** */
-            short? testWrapper(short? x);
-
-            /** */
-            int? testWrapper(int? x);
-
-            /** */
-            long? testWrapper(long? x);
-
-            /** */
-            float? testWrapper(float? x);
-
-            /** */
-            double? testWrapper(double? x);
-
-            /** */
-            char? testWrapper(char? x);
-
-            /** */
-            bool? testWrapper(bool? x);
-
-            /** */
-            byte[] testArray(byte[] x);
-
-            /** */
-            short[] testArray(short[] x);
-
-            /** */
-            int[] testArray(int[] x);
-
-            /** */
-            long[] testArray(long[] x);
-
-            /** */
-            float[] testArray(float[] x);
-
-            /** */
-            double[] testArray(double[] x);
-
-            /** */
-            char[] testArray(char[] x);
-
-            /** */
-            string[] testArray(string[] x);
-
-            /** */
-            bool[] testArray(bool[] x);
-
-            /** */
-            DateTime?[] testArray(DateTime?[] x);
-
-            /** */
-            Guid?[] testArray(Guid?[] x);
-
-            /** */
-            int test(int x, string y);
-
-            /** */
-            int test(string x, int y);
-
-            /** */
-            int? testNull(int? x);
-
-            /** */
-            DateTime? testNullTimestamp(DateTime? x);
-
-            /** */
-            Guid? testNullUUID(Guid? x);
-
-            /** */
-            int testParams(params object[] args);
-
-            /** */
-            PlatformComputeBinarizable testBinarizable(PlatformComputeBinarizable x);
-
-            /** */
-            object[] testBinarizableArrayOfObjects(object[] x);
-
-            /** */
-            IBinaryObject[] testBinaryObjectArray(IBinaryObject[] x);
-
-            /** */
-            PlatformComputeBinarizable[] testBinarizableArray(PlatformComputeBinarizable[] x);
-
-            /** */
-            ICollection testBinarizableCollection(ICollection x);
-
-            /** */
-            IBinaryObject testBinaryObject(IBinaryObject x);
-        }
-
-        /// <summary>
         /// Interop class.
         /// </summary>
         public class PlatformComputeBinarizable
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs
index cb5fe50..494fb13 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs
@@ -649,6 +649,23 @@ namespace Apache.Ignite.Core.Tests
                 }
             }
         }
+        
+        /// <summary>
+        /// Deploys the Java service.
+        /// </summary>
+        public static string DeployJavaService(IIgnite ignite)
+        {
+            const string serviceName = "javaService";
+            
+            ignite.GetCompute()
+                .ExecuteJavaTask<object>("org.apache.ignite.platform.PlatformDeployServiceTask", serviceName);
+            
+            var services = ignite.GetServices();
+
+            WaitForCondition(() => services.GetServiceDescriptors().Any(x => x.Name == serviceName), 1000);
+
+            return serviceName;
+        }
 
         /// <summary>
         /// Logs to test progress. Produces immediate console output on .NET Core.
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 beb315d..5dfc27b 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
@@ -69,6 +69,7 @@
     <Compile Include="Client\IClientConnection.cs" />
     <Compile Include="Client\IgniteClientConfigurationSection.cs" />
     <Compile Include="Client\ISslStreamFactory.cs" />
+    <Compile Include="Client\Services\IServicesClient.cs" />
     <Compile Include="Client\SslStreamFactory.cs" />
     <Compile Include="Cluster\IBaselineNode.cs" />
     <Compile Include="Common\IgniteProductVersion.cs" />
@@ -124,6 +125,7 @@
     <Compile Include="Impl\Client\Compute\ComputeClient.cs" />
     <Compile Include="Impl\Client\Compute\ComputeClientFlags.cs" />
     <Compile Include="Impl\Client\Endpoint.cs" />
+    <Compile Include="Impl\Client\Services\ServicesClient.cs" />
     <Compile Include="Impl\Client\SocketEndpoint.cs" />
     <Compile Include="Impl\Common\PlatformType.cs" />
     <Compile Include="Impl\Common\TaskRunner.cs" />
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Client/Compute/IComputeClient.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Client/Compute/IComputeClient.cs
index bae8c24..ce85454 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Client/Compute/IComputeClient.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Client/Compute/IComputeClient.cs
@@ -28,6 +28,11 @@ namespace Apache.Ignite.Core.Client.Compute
     public interface IComputeClient
     {
         /// <summary>
+        /// Gets the cluster group for this <see cref="IComputeClient"/> instance.
+        /// </summary>
+        IClientClusterGroup ClusterGroup { get; }
+
+        /// <summary>
         /// Executes Java task by class name.
         /// </summary>
         /// <param name="taskName">Java task name.</param>
@@ -61,20 +66,20 @@ namespace Apache.Ignite.Core.Client.Compute
         /// <param name="timeout">Timeout.</param>
         /// <returns>New Compute instance with timeout.</returns>
         IComputeClient WithTimeout(TimeSpan timeout);
-        
+
         /// <summary>
         /// Returns a new instance of <see cref="IComputeClient"/> with disabled failover.
         /// When failover is disabled, compute jobs won't be retried in case of node crashes.
         /// </summary>
         /// <returns>New Compute instance with disabled failover.</returns>
         IComputeClient WithNoFailover();
-        
+
         /// <summary>
         /// Returns a new instance of <see cref="IComputeClient"/> with disabled result cache.
         /// </summary>
         /// <returns>New Compute instance with disabled result cache.</returns>
         IComputeClient WithNoResultCache();
-        
+
         /// <summary>
         /// Returns a new instance of <see cref="IComputeClient"/> with binary mode enabled:
         /// Java task argument (on server side) and result (on client side) won't be deserialized.
@@ -82,4 +87,4 @@ namespace Apache.Ignite.Core.Client.Compute
         /// <returns>New Compute instance with binary mode enabled.</returns>
         IComputeClient WithKeepBinary();
     }
-}
\ No newline at end of file
+}
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Client/IClientClusterGroup.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Client/IClientClusterGroup.cs
index 648e3dc..98203de 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Client/IClientClusterGroup.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Client/IClientClusterGroup.cs
@@ -21,6 +21,7 @@ namespace Apache.Ignite.Core.Client
     using System.Collections.Generic;
     using System.Diagnostics.CodeAnalysis;
     using Apache.Ignite.Core.Client.Compute;
+    using Apache.Ignite.Core.Client.Services;
 
     /// <summary>
     /// Defines grid projection which represents a common functionality over a group of nodes.
@@ -31,13 +32,13 @@ namespace Apache.Ignite.Core.Client
     /// shows how to create grid projections:
     /// <code>
     /// var g = Ignition.StartClient().GetCluster();
-    /// 
+    ///
     /// // Projection over .NET nodes.
     /// var remoteNodes = g.ForDotNet();
-    /// 
+    ///
     /// // Projection over server nodes.
     /// var randomNode = g.ForServers();
-    /// 
+    ///
     /// // Projection over all nodes that have user attribute "group" set to value "worker".
     /// var workerNodes = g.ForAttribute("group", "worker");
     /// </code>
@@ -83,7 +84,7 @@ namespace Apache.Ignite.Core.Client
         /// Gets a node for given ID from this grid projection.
         /// </summary>
         /// <param name="id">Node ID.</param>
-        /// <returns>Node with given ID from this projection or null if such node does not 
+        /// <returns>Node with given ID from this projection or null if such node does not
         /// exist in this projection.</returns>
         IClientClusterNode GetNode(Guid id);
 
@@ -101,5 +102,13 @@ namespace Apache.Ignite.Core.Client
         /// <returns>Compute instance over this grid projection.</returns>
         [SuppressMessage("Microsoft.Design", "CA1024:UsePropertiesWhereAppropriate", Justification = "Semantics.")]
         IComputeClient GetCompute();
+
+        /// <summary>
+        /// Gets services functionality over this grid projection. All operations
+        /// on the returned services instance will only include nodes from this projection.
+        /// </summary>
+        /// <returns>Services instance over this grid projection.</returns>
+        [SuppressMessage("Microsoft.Design", "CA1024:UsePropertiesWhereAppropriate", Justification = "Semantics.")]
+        IServicesClient GetServices();
     }
 }
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Client/IIgniteClient.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Client/IIgniteClient.cs
index 9a18d27..68a4c82 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Client/IIgniteClient.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Client/IIgniteClient.cs
@@ -24,6 +24,7 @@ namespace Apache.Ignite.Core.Client
     using Apache.Ignite.Core.Binary;
     using Apache.Ignite.Core.Client.Cache;
     using Apache.Ignite.Core.Client.Compute;
+    using Apache.Ignite.Core.Client.Services;
 
     /// <summary>
     /// Main entry point for Ignite Thin Client APIs.
@@ -98,7 +99,7 @@ namespace Apache.Ignite.Core.Client
         IClientCluster GetCluster();
 
         /// <summary>
-        /// Destroys dynamically created (with <see cref="CreateCache{TK,TV}(string)"/> or 
+        /// Destroys dynamically created (with <see cref="CreateCache{TK,TV}(string)"/> or
         /// <see cref="GetOrCreateCache{TK,TV}(string)"/>) cache.
         /// </summary>
         /// <param name="name">The name of the cache to stop.</param>
@@ -141,5 +142,10 @@ namespace Apache.Ignite.Core.Client
         /// Gets the compute API.
         /// </summary>
         IComputeClient GetCompute();
+
+        /// <summary>
+        /// Gets the services API.
+        /// </summary>
+        IServicesClient GetServices();
     }
 }
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Client/Services/IServicesClient.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Client/Services/IServicesClient.cs
new file mode 100644
index 0000000..49c64fd
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Client/Services/IServicesClient.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.Client.Services
+{
+    /// <summary>
+    /// Ignite distributed services client.
+    /// </summary>
+    public interface IServicesClient
+    {
+        /// <summary>
+        /// Gets the cluster group for this <see cref="IServicesClient"/> instance.
+        /// </summary>
+        IClientClusterGroup ClusterGroup { get; }
+
+        /// <summary>
+        /// Gets a proxy for the service with the specified name.
+        /// <para />
+        /// Note: service proxies are not "sticky" - there is no guarantee that all calls will be made to the same
+        /// remote service instance.
+        /// </summary>
+        /// <typeparam name="T">Service type.</typeparam>
+        /// <param name="serviceName">Service name.</param>
+        /// <returns>Proxy object that forwards all member calls to a remote Ignite service.</returns>
+        T GetServiceProxy<T>(string serviceName) where T : class;
+
+        /// <summary>
+        /// Returns an instance with binary mode enabled.
+        /// Service method results will be kept in binary form.
+        /// </summary>
+        /// <returns>Instance with binary mode enabled.</returns>
+        IServicesClient WithKeepBinary();
+
+        /// <summary>
+        /// Returns an instance with server-side binary mode enabled.
+        /// Service method arguments will be kept in binary form.
+        /// </summary>
+        /// <returns>Instance with server-side binary mode enabled.</returns>
+        IServicesClient WithServerKeepBinary();
+    }
+}
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientOp.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientOp.cs
index 486a9eb0..238058f 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientOp.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientOp.cs
@@ -83,6 +83,9 @@ namespace Apache.Ignite.Core.Impl.Client
 
         // Compute.
         ComputeTaskExecute = 6000,
-        ComputeTaskFinished = 6001
+        ComputeTaskFinished = 6001,
+
+        // Services.
+        ServiceInvoke = 7000
     }
 }
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/Cluster/ClientClusterGroup.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/Cluster/ClientClusterGroup.cs
index a47a694..d5f9ead 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/Cluster/ClientClusterGroup.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/Cluster/ClientClusterGroup.cs
@@ -27,7 +27,9 @@ namespace Apache.Ignite.Core.Impl.Client.Cluster
     using Apache.Ignite.Core.Impl.Binary;
     using System.Linq;
     using Apache.Ignite.Core.Client.Compute;
+    using Apache.Ignite.Core.Client.Services;
     using Apache.Ignite.Core.Impl.Client.Compute;
+    using Apache.Ignite.Core.Impl.Client.Services;
     using Apache.Ignite.Core.Impl.Common;
 
     /// <summary>
@@ -75,7 +77,7 @@ namespace Apache.Ignite.Core.Impl.Client.Cluster
         /// <param name="ignite">Ignite.</param>
         /// <param name="projection">Projection.</param>
         /// <param name="predicate">Predicate.</param>
-        private ClientClusterGroup(IgniteClient ignite, 
+        private ClientClusterGroup(IgniteClient ignite,
             ClientClusterGroupProjection projection, Func<IClientClusterNode, bool> predicate = null)
         {
             Debug.Assert(ignite != null);
@@ -143,6 +145,12 @@ namespace Apache.Ignite.Core.Impl.Client.Cluster
             return new ComputeClient(_ignite, ComputeClientFlags.None, TimeSpan.Zero, this);
         }
 
+        /** <inheritDoc /> */
+        public IServicesClient GetServices()
+        {
+            return new ServicesClient(_ignite, this);
+        }
+
         /// <summary>
         /// Refresh projection nodes.
         /// </summary>
@@ -213,7 +221,7 @@ namespace Apache.Ignite.Core.Impl.Client.Cluster
 
             var nodeIds = new Guid[nodesCount];
             var stream = ((BinaryReader) reader).Stream;
-            
+
             for (int i = 0; i < nodesCount; i++)
             {
                 nodeIds[i] = BinaryUtils.ReadGuid(stream);
@@ -258,7 +266,7 @@ namespace Apache.Ignite.Core.Impl.Client.Cluster
                     unknownNodes.Add(nodeId);
                 }
             }
-            
+
             if (unknownNodes.Count > 0)
             {
                 RequestRemoteNodesDetails(unknownNodes);
@@ -279,7 +287,7 @@ namespace Apache.Ignite.Core.Impl.Client.Cluster
                     BinaryUtils.WriteGuid(id, ctx.Stream);
                 }
             };
-            
+
             Func<ClientResponseContext, bool> readFunc = ctx =>
             {
                 var cnt = ctx.Stream.ReadInt();
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/Compute/ComputeClient.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/Compute/ComputeClient.cs
index 6e1fdca..d052fdd 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/Compute/ComputeClient.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/Compute/ComputeClient.cs
@@ -121,6 +121,12 @@ namespace Apache.Ignite.Core.Impl.Client.Compute
             return SetFlag(ComputeClientFlags.KeepBinary);
         }
 
+        /** <inheritdoc /> */
+        public IClientClusterGroup ClusterGroup
+        {
+            get { return _clusterGroup ?? _ignite.GetCluster(); }
+        }
+
         /// <summary>
         /// Returns a new instance with the given flag enabled, or this instance if the flag is already present.
         /// </summary>
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/IgniteClient.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/IgniteClient.cs
index 183f8f9..cad5ea1 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/IgniteClient.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/IgniteClient.cs
@@ -29,6 +29,7 @@ namespace Apache.Ignite.Core.Impl.Client
     using Apache.Ignite.Core.Client;
     using Apache.Ignite.Core.Client.Cache;
     using Apache.Ignite.Core.Client.Compute;
+    using Apache.Ignite.Core.Client.Services;
     using Apache.Ignite.Core.Datastream;
     using Apache.Ignite.Core.Impl.Binary;
     using Apache.Ignite.Core.Impl.Cache;
@@ -36,6 +37,7 @@ namespace Apache.Ignite.Core.Impl.Client
     using Apache.Ignite.Core.Impl.Client.Cache;
     using Apache.Ignite.Core.Impl.Client.Cluster;
     using Apache.Ignite.Core.Impl.Client.Compute;
+    using Apache.Ignite.Core.Impl.Client.Services;
     using Apache.Ignite.Core.Impl.Cluster;
     using Apache.Ignite.Core.Impl.Common;
     using Apache.Ignite.Core.Impl.Handle;
@@ -71,6 +73,9 @@ namespace Apache.Ignite.Core.Impl.Client
         /** Compute. */
         private readonly ComputeClient _compute;
 
+        /** Services. */
+        private readonly IServicesClient _services;
+
         /// <summary>
         /// Initializes a new instance of the <see cref="IgniteClient"/> class.
         /// </summary>
@@ -95,6 +100,8 @@ namespace Apache.Ignite.Core.Impl.Client
             _cluster = new ClientCluster(this);
 
             _compute = new ComputeClient(this, ComputeClientFlags.None, TimeSpan.Zero, null);
+
+            _services = new ServicesClient(this);
         }
 
         /// <summary>
@@ -246,6 +253,12 @@ namespace Apache.Ignite.Core.Impl.Client
         }
 
         /** <inheritDoc /> */
+        public IServicesClient GetServices()
+        {
+            return _services;
+        }
+
+        /** <inheritDoc /> */
         public IBinaryProcessor BinaryProcessor
         {
             get { return _binProc; }
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/Services/ServicesClient.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/Services/ServicesClient.cs
new file mode 100644
index 0000000..a650b21
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/Services/ServicesClient.cs
@@ -0,0 +1,157 @@
+/*
+ * 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.Client.Services
+{
+    using System;
+    using System.Diagnostics;
+    using System.Reflection;
+    using Apache.Ignite.Core.Client;
+    using Apache.Ignite.Core.Client.Services;
+    using Apache.Ignite.Core.Impl.Binary;
+    using Apache.Ignite.Core.Impl.Common;
+    using Apache.Ignite.Core.Impl.Services;
+
+    /// <summary>
+    /// Services client.
+    /// </summary>
+    internal class ServicesClient : IServicesClient
+    {
+        /** */
+        [Flags]
+        private enum ServiceFlags : byte
+        {
+            KeepBinary = 1,
+
+            // ReSharper disable once UnusedMember.Local
+            HasParameterTypes = 2
+        }
+
+        /** */
+        private readonly IgniteClient _ignite;
+
+        /** */
+        private readonly IClientClusterGroup _clusterGroup;
+
+        /** */
+        private readonly bool _keepBinary;
+
+        /** */
+        private readonly bool _serverKeepBinary;
+
+        /** */
+        private readonly TimeSpan _timeout;
+
+        /// <summary>
+        /// Initializes a new instance of <see cref="ServicesClient"/> class.
+        /// </summary>
+        public ServicesClient(
+            IgniteClient ignite,
+            IClientClusterGroup clusterGroup = null,
+            bool keepBinary = false,
+            bool serverKeepBinary = false,
+            TimeSpan timeout = default(TimeSpan))
+        {
+            Debug.Assert(ignite != null);
+
+            _ignite = ignite;
+            _clusterGroup = clusterGroup;
+            _keepBinary = keepBinary;
+            _serverKeepBinary = serverKeepBinary;
+            _timeout = timeout;
+        }
+
+        /** <inheritdoc /> */
+        public IClientClusterGroup ClusterGroup
+        {
+            get { return _clusterGroup ?? _ignite.GetCluster(); }
+        }
+
+        /** <inheritdoc /> */
+        public T GetServiceProxy<T>(string serviceName) where T : class
+        {
+            IgniteArgumentCheck.NotNullOrEmpty(serviceName, "name");
+
+            return ServiceProxyFactory<T>.CreateProxy((method, args) => InvokeProxyMethod(serviceName, method, args));
+        }
+
+        /** <inheritdoc /> */
+        public IServicesClient WithKeepBinary()
+        {
+            return new ServicesClient(_ignite, _clusterGroup, true, _serverKeepBinary, _timeout);
+        }
+
+        /** <inheritdoc /> */
+        public IServicesClient WithServerKeepBinary()
+        {
+            return new ServicesClient(_ignite, _clusterGroup, _keepBinary, true, _timeout);
+        }
+
+        /// <summary>
+        /// Invokes the proxy method.
+        /// </summary>
+        private object InvokeProxyMethod(string serviceName, MethodBase method, object[] args)
+        {
+            return _ignite.Socket.DoOutInOp(
+                ClientOp.ServiceInvoke,
+                ctx =>
+                {
+                    var w = ctx.Writer;
+
+                    w.WriteString(serviceName);
+                    w.WriteByte(_serverKeepBinary ? (byte) ServiceFlags.KeepBinary : (byte) 0);
+                    w.WriteLong((long) _timeout.TotalMilliseconds);
+
+                    if (_clusterGroup != null)
+                    {
+                        var nodes = _clusterGroup.GetNodes();
+                        if (nodes.Count == 0)
+                        {
+                            throw new IgniteClientException("Cluster group is empty");
+                        }
+
+                        w.WriteInt(nodes.Count);
+
+                        foreach (var node in nodes)
+                        {
+                            BinaryUtils.WriteGuid(node.Id, ctx.Stream);
+                        }
+                    }
+                    else
+                    {
+                        w.WriteInt(0);
+                    }
+
+                    w.WriteString(method.Name);
+
+                    w.WriteInt(args.Length);
+                    foreach (var arg in args)
+                    {
+                        w.WriteObjectDetached(arg);
+                    }
+                },
+                ctx =>
+                {
+                    var reader = _keepBinary
+                        ? ctx.Marshaller.StartUnmarshal(ctx.Stream, BinaryMode.ForceBinary)
+                        : ctx.Reader;
+
+                    return reader.ReadObject<object>();
+                });
+        }
+    }
+}
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/DelegateConverter.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/DelegateConverter.cs
index 6daea7c..256d3e7 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/DelegateConverter.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/DelegateConverter.cs
@@ -26,10 +26,12 @@ namespace Apache.Ignite.Core.Impl.Common
     using System.Reflection;
     using System.Reflection.Emit;
     using Apache.Ignite.Core.Binary;
+    using Apache.Ignite.Core.Common;
 
     /// <summary>
     /// Converts generic and non-generic delegates.
     /// </summary>
+    [CLSCompliant(false)]
     public static class DelegateConverter
     {
         /** */
@@ -39,9 +41,30 @@ namespace Apache.Ignite.Core.Impl.Common
         private static readonly MethodInfo ReadObjectMethod = typeof (IBinaryRawReader).GetMethod("ReadObject");
 
         /** */
-        private static readonly MethodInfo ConvertArrayMethod = typeof(DelegateConverter).GetMethod("ConvertArray",
+        public static readonly MethodInfo ConvertArrayMethod = typeof(DelegateConverter).GetMethod(
+            "ConvertArray",
             BindingFlags.Static | BindingFlags.NonPublic);
 
+        /** */
+        public static readonly MethodInfo ConvertToSbyteArrayMethod = typeof(DelegateConverter).GetMethod(
+            "ConvertToSbyteArray",
+            BindingFlags.Static | BindingFlags.Public);
+
+        /** */
+        public static readonly MethodInfo ConvertToUshortArrayMethod = typeof(DelegateConverter).GetMethod(
+            "ConvertToUshortArray",
+            BindingFlags.Static | BindingFlags.Public);
+
+        /** */
+        public static readonly MethodInfo ConvertToUintArrayMethod = typeof(DelegateConverter).GetMethod(
+            "ConvertToUintArray",
+            BindingFlags.Static | BindingFlags.Public);
+
+        /** */
+        public static readonly MethodInfo ConvertToUlongArrayMethod = typeof(DelegateConverter).GetMethod(
+            "ConvertToUlongArray",
+            BindingFlags.Static | BindingFlags.Public);
+
         /// <summary>
         /// Compiles a function without arguments.
         /// </summary>
@@ -532,13 +555,61 @@ namespace Apache.Ignite.Core.Impl.Common
         /// </summary>
         private static Expression Convert(Expression value, Type targetType)
         {
-            if (targetType.IsArray && targetType.GetElementType() != typeof(object))
+            if (targetType.IsArray)
             {
-                var convertMethod = ConvertArrayMethod.MakeGenericMethod(targetType.GetElementType());
+                var elType = targetType.GetElementType();
+                Debug.Assert(elType != null);
+
+                if (elType == typeof(sbyte))
+                {
+                    return Expression.Call(null, ConvertToSbyteArrayMethod, value);
+                }
 
-                var objArray = Expression.Convert(value, typeof(object[]));
+                if (elType == typeof(ushort))
+                {
+                    return Expression.Call(null, ConvertToUshortArrayMethod, value);
+                }
 
-                return Expression.Call(null, convertMethod, objArray);
+                if (elType == typeof(uint))
+                {
+                    return Expression.Call(null, ConvertToUintArrayMethod, value);
+                }
+
+                if (elType == typeof(ulong))
+                {
+                    return Expression.Call(null, ConvertToUlongArrayMethod, value);
+                }
+
+                if (elType != typeof(object))
+                {
+                    var convertMethod = ConvertArrayMethod.MakeGenericMethod(targetType.GetElementType());
+
+                    return Expression.Call(null, convertMethod, value);
+                }
+
+                return Expression.Convert(value, targetType);
+            }
+
+            // For byte/sbyte and the like, simple Convert fails
+            // E.g. the following does not work:   (sbyte)(object)((byte)1)
+            // But this does:                      (sbyte)(byte)(object)((byte)1)
+            // So for every "unsupported" type like sbyte, ushort, uint, ulong
+            // we have to do an additional conversion
+            if (targetType == typeof(sbyte))
+            {
+                value = Expression.Convert(value, typeof(byte));
+            }
+            else if (targetType == typeof(ushort))
+            {
+                value = Expression.Convert(value, typeof(short));
+            }
+            else if (targetType == typeof(uint))
+            {
+                value = Expression.Convert(value, typeof(int));
+            }
+            else if (targetType == typeof(ulong))
+            {
+                value = Expression.Convert(value, typeof(long));
             }
 
             return Expression.Convert(value, targetType);
@@ -548,8 +619,9 @@ namespace Apache.Ignite.Core.Impl.Common
         /// Converts object array to typed array.
         /// </summary>
         // ReSharper disable once UnusedMember.Local (used by reflection).
-        private static T[] ConvertArray<T>(object[] arr)
+        private static T[] ConvertArray<T>(object arrObj)
         {
+            var arr = arrObj as Array;
             if (arr == null)
             {
                 return null;
@@ -561,5 +633,64 @@ namespace Apache.Ignite.Core.Impl.Common
 
             return res;
         }
+
+        /// <summary>
+        /// Converts to sbyte array.
+        /// </summary>
+        // ReSharper disable once UnusedMember.Global
+        public static sbyte[] ConvertToSbyteArray(object arrObj)
+        {
+            return ConvertValueTypeArray<byte, sbyte>(arrObj, 1);
+        }
+
+        /// <summary>
+        /// Converts to ushort array.
+        /// </summary>
+        // ReSharper disable once UnusedMember.Global
+        public static ushort[] ConvertToUshortArray(object arrObj)
+        {
+            return ConvertValueTypeArray<short, ushort>(arrObj, 2);
+        }
+
+        /// <summary>
+        /// Converts to uint array.
+        /// </summary>
+        // ReSharper disable once UnusedMember.Global
+        public static uint[] ConvertToUintArray(object arrObj)
+        {
+            return ConvertValueTypeArray<int, uint>(arrObj, 4);
+        }
+
+        /// <summary>
+        /// Converts to ulong array.
+        /// </summary>
+        // ReSharper disable once UnusedMember.Global
+        public static ulong[] ConvertToUlongArray(object arrObj)
+        {
+            return ConvertValueTypeArray<long, ulong>(arrObj, 8);
+        }
+
+        /// <summary>
+        /// Converts value type array to another type using direct copy.
+        /// </summary>
+        private static T[] ConvertValueTypeArray<TFrom, T>(object arrObj, int elementSize)
+        {
+            if (arrObj == null)
+            {
+                return null;
+            }
+
+            var arr = arrObj as TFrom[];
+            if (arr == null)
+            {
+                throw new IgniteException(string.Format("Can't convert '{0}' to '{1}'", arrObj.GetType(), typeof(T[])));
+            }
+
+            var res = new T[arr.Length];
+
+            Buffer.BlockCopy(arr, 0, res, 0, arr.Length * elementSize);
+
+            return res;
+        }
     }
 }
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Services/ServiceProxyTypeGenerator.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Services/ServiceProxyTypeGenerator.cs
index 8c84ccd..0d332a9 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Services/ServiceProxyTypeGenerator.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Services/ServiceProxyTypeGenerator.cs
@@ -24,6 +24,7 @@ namespace Apache.Ignite.Core.Impl.Services
     using System.Reflection;
     using System.Reflection.Emit;
     using Apache.Ignite.Core.Impl.Binary;
+    using Apache.Ignite.Core.Impl.Common;
     using ProxyAction = System.Func<System.Reflection.MethodBase, object[], object>;
 
     /// <summary>
@@ -276,8 +277,7 @@ namespace Apache.Ignite.Core.Impl.Services
             // Load result.
             if (method.ReturnType != typeof(void))
             {
-                if (method.ReturnType.IsValueType)
-                    gen.Emit(OpCodes.Unbox_Any, method.ReturnType);
+                EmitReturn(gen, method);
             }
             else
             {
@@ -289,6 +289,53 @@ namespace Apache.Ignite.Core.Impl.Services
         }
 
         /// <summary>
+        /// Emits the returned value, converting as necessary.
+        /// </summary>
+        private static void EmitReturn(ILGenerator gen, MethodInfo method)
+        {
+            if (method.ReturnType == typeof(sbyte))
+            {
+                gen.Emit(OpCodes.Unbox_Any, typeof(byte));
+                gen.Emit(OpCodes.Conv_I1);
+            }
+            else if (method.ReturnType == typeof(ushort))
+            {
+                gen.Emit(OpCodes.Unbox_Any, typeof(short));
+                gen.Emit(OpCodes.Conv_U2);
+            }
+            else if (method.ReturnType == typeof(uint))
+            {
+                gen.Emit(OpCodes.Unbox_Any, typeof(int));
+                gen.Emit(OpCodes.Conv_U4);
+            }
+            else if (method.ReturnType == typeof(ulong))
+            {
+                gen.Emit(OpCodes.Unbox_Any, typeof(long));
+                gen.Emit(OpCodes.Conv_U8);
+            }
+            else if (method.ReturnType == typeof(sbyte[]))
+            {
+                gen.Emit(OpCodes.Call, DelegateConverter.ConvertToSbyteArrayMethod);
+            }
+            else if (method.ReturnType == typeof(ushort[]))
+            {
+                gen.Emit(OpCodes.Call, DelegateConverter.ConvertToUshortArrayMethod);
+            }
+            else if (method.ReturnType == typeof(uint[]))
+            {
+                gen.Emit(OpCodes.Call, DelegateConverter.ConvertToUintArrayMethod);
+            }
+            else if (method.ReturnType == typeof(ulong[]))
+            {
+                gen.Emit(OpCodes.Call, DelegateConverter.ConvertToUlongArrayMethod);
+            }
+            else if (method.ReturnType.IsValueType)
+            {
+                gen.Emit(OpCodes.Unbox_Any, method.ReturnType);
+            }
+        }
+
+        /// <summary>
         /// Proxy build state.
         /// </summary>
         private class ProxyBuildContext