You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by GitBox <gi...@apache.org> on 2020/06/10 15:49:06 UTC

[GitHub] [ignite] ptupitsyn commented on a change in pull request #7908: IGNITE-13033 Java thin client: Service invocation

ptupitsyn commented on a change in pull request #7908:
URL: https://github.com/apache/ignite/pull/7908#discussion_r438206235



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientServicesImpl.java
##########
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.client.thin;
+
+import java.lang.reflect.InvocationHandler;
+import java.lang.reflect.Method;
+import java.lang.reflect.Proxy;
+import java.util.Collection;
+import java.util.UUID;
+import org.apache.ignite.client.ClientClusterGroup;
+import org.apache.ignite.client.ClientException;
+import org.apache.ignite.client.ClientServices;
+import org.apache.ignite.internal.binary.BinaryRawWriterEx;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.A;
+import org.apache.ignite.platform.PlatformServiceMethod;
+
+/**
+ * Implementation of {@link ClientServices}.
+ */
+class ClientServicesImpl implements ClientServices {
+    /** Channel. */
+    private final ReliableChannel ch;
+
+    /** Binary marshaller. */
+    private final ClientBinaryMarshaller marsh;
+
+    /** Utils for serialization/deserialization. */
+    private final ClientUtils utils;
+
+    /** Cluster group. */
+    private final ClientClusterGroupImpl grp;
+
+    /** Constructor. */
+    ClientServicesImpl(ReliableChannel ch, ClientBinaryMarshaller marsh, ClientClusterGroupImpl grp) {
+        this.ch = ch;
+        this.marsh = marsh;
+        this.grp = grp;
+
+        utils = new ClientUtils(marsh);
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClientClusterGroup clusterGroup() {
+        return grp;
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> T serviceProxy(String name, Class<? super T> svcItf) {
+        return serviceProxy(name, svcItf, 0);
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> T serviceProxy(String name, Class<? super T> svcItf, long timeout) {
+        A.notNullOrEmpty(name, "name");
+        A.notNull(svcItf, "svcItf");
+
+        Collection<UUID> nodeIds = grp.nodeIds();
+
+        if (nodeIds != null && nodeIds.isEmpty())
+            throw new ClientException("Cluster group is empty.");
+
+        return (T)Proxy.newProxyInstance(svcItf.getClassLoader(), new Class[] {svcItf},
+            new ServiceInvocationHandler<>(name, timeout, nodeIds));
+    }
+
+    /**
+     * Gets services facade over the specified cluster group.
+     *
+     * @param grp Cluster group.
+     */
+    ClientServices withClusterGroup(ClientClusterGroupImpl grp) {
+        A.notNull(grp, "grp");
+
+        return new ClientServicesImpl(ch, marsh, grp);
+    }
+
+    /**
+     * Service invocation handler.
+     */
+    private class ServiceInvocationHandler<T> implements InvocationHandler {
+        /** Service name. */
+        private final String name;
+
+        /** Timeout. */
+        private final long timeout;
+
+        /** Node IDs. */
+        private final Collection<UUID> nodeIds;
+
+        /**
+         * @param name Service name.
+         * @param timeout Timeout.
+         */
+        private ServiceInvocationHandler(String name, long timeout, Collection<UUID> nodeIds) {
+            this.name = name;
+            this.timeout = timeout;
+            this.nodeIds = nodeIds;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object invoke(Object proxy, Method method, Object[] args) throws Throwable {
+            try {
+                return ch.service(ClientOperation.SERVICE_INVOKE,
+                    req -> writeServiceInvokeRequest(req, method, args),
+                    res -> utils.readObject(res.in(), false)
+                );
+            }
+            catch (ClientError e) {
+                throw new ClientException(e);
+            }
+        }
+
+        /**
+         * @param ch Payload output channel.
+         */
+        private void writeServiceInvokeRequest(PayloadOutputChannel ch, Method method, Object[] args) {
+            ch.clientChannel().protocolCtx().checkFeatureSupported(ProtocolBitmaskFeature.SERVICE_INVOKE);
+
+            try (BinaryRawWriterEx writer = utils.createBinaryWriter(ch.out())) {
+                writer.writeString(name);
+                writer.writeByte((byte)0); // Flags.
+                writer.writeLong(timeout);
+
+                if (nodeIds == null)
+                    writer.writeInt(0);
+                else {
+                    writer.writeInt(nodeIds.size());
+
+                    for (UUID nodeId : nodeIds)
+                        writer.writeUuid(nodeId);
+                }
+
+                PlatformServiceMethod ann = method.getDeclaredAnnotation(PlatformServiceMethod.class);
+
+                writer.writeString(ann != null ? ann.value() : method.getName());
+
+                Class<?>[] paramTypes = method.getParameterTypes();
+                int[] paramTypeIds = new int[paramTypes.length];
+
+                for (int i = 0; i < paramTypes.length; i++)
+                    paramTypeIds[i] = marsh.context().typeId(paramTypes[i].getName());
+
+                writer.writeIntArray(paramTypeIds);
+
+                if (F.isEmpty(args))
+                    writer.writeInt(0);
+                else {
+                    writer.writeInt(args.length);

Review comment:
       Unnecessary: we already have the length above for paramTypeIds.
   I missed this in IEP. In my opinion, it is better to do the following:
   - Write 'hasTypes' boolean value
   - Write count once
   - In a loop, write argument value + argument type (if hasTypes)
   
   It seems to be easier to decode on the consumer side. Thoughts?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientServicesImpl.java
##########
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.client.thin;
+
+import java.lang.reflect.InvocationHandler;
+import java.lang.reflect.Method;
+import java.lang.reflect.Proxy;
+import java.util.Collection;
+import java.util.UUID;
+import org.apache.ignite.client.ClientClusterGroup;
+import org.apache.ignite.client.ClientException;
+import org.apache.ignite.client.ClientServices;
+import org.apache.ignite.internal.binary.BinaryRawWriterEx;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.A;
+import org.apache.ignite.platform.PlatformServiceMethod;
+
+/**
+ * Implementation of {@link ClientServices}.
+ */
+class ClientServicesImpl implements ClientServices {
+    /** Channel. */
+    private final ReliableChannel ch;
+
+    /** Binary marshaller. */
+    private final ClientBinaryMarshaller marsh;
+
+    /** Utils for serialization/deserialization. */
+    private final ClientUtils utils;
+
+    /** Cluster group. */
+    private final ClientClusterGroupImpl grp;
+
+    /** Constructor. */
+    ClientServicesImpl(ReliableChannel ch, ClientBinaryMarshaller marsh, ClientClusterGroupImpl grp) {
+        this.ch = ch;
+        this.marsh = marsh;
+        this.grp = grp;
+
+        utils = new ClientUtils(marsh);
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClientClusterGroup clusterGroup() {
+        return grp;
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> T serviceProxy(String name, Class<? super T> svcItf) {
+        return serviceProxy(name, svcItf, 0);
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> T serviceProxy(String name, Class<? super T> svcItf, long timeout) {
+        A.notNullOrEmpty(name, "name");
+        A.notNull(svcItf, "svcItf");
+
+        Collection<UUID> nodeIds = grp.nodeIds();
+
+        if (nodeIds != null && nodeIds.isEmpty())
+            throw new ClientException("Cluster group is empty.");
+
+        return (T)Proxy.newProxyInstance(svcItf.getClassLoader(), new Class[] {svcItf},
+            new ServiceInvocationHandler<>(name, timeout, nodeIds));
+    }
+
+    /**
+     * Gets services facade over the specified cluster group.
+     *
+     * @param grp Cluster group.
+     */
+    ClientServices withClusterGroup(ClientClusterGroupImpl grp) {
+        A.notNull(grp, "grp");
+
+        return new ClientServicesImpl(ch, marsh, grp);
+    }
+
+    /**
+     * Service invocation handler.
+     */
+    private class ServiceInvocationHandler<T> implements InvocationHandler {
+        /** Service name. */
+        private final String name;
+
+        /** Timeout. */
+        private final long timeout;
+
+        /** Node IDs. */
+        private final Collection<UUID> nodeIds;
+
+        /**
+         * @param name Service name.
+         * @param timeout Timeout.
+         */
+        private ServiceInvocationHandler(String name, long timeout, Collection<UUID> nodeIds) {
+            this.name = name;
+            this.timeout = timeout;
+            this.nodeIds = nodeIds;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object invoke(Object proxy, Method method, Object[] args) throws Throwable {
+            try {
+                return ch.service(ClientOperation.SERVICE_INVOKE,
+                    req -> writeServiceInvokeRequest(req, method, args),
+                    res -> utils.readObject(res.in(), false)
+                );
+            }
+            catch (ClientError e) {
+                throw new ClientException(e);
+            }
+        }
+
+        /**
+         * @param ch Payload output channel.
+         */
+        private void writeServiceInvokeRequest(PayloadOutputChannel ch, Method method, Object[] args) {
+            ch.clientChannel().protocolCtx().checkFeatureSupported(ProtocolBitmaskFeature.SERVICE_INVOKE);
+
+            try (BinaryRawWriterEx writer = utils.createBinaryWriter(ch.out())) {
+                writer.writeString(name);
+                writer.writeByte((byte)0); // Flags.
+                writer.writeLong(timeout);
+
+                if (nodeIds == null)
+                    writer.writeInt(0);
+                else {
+                    writer.writeInt(nodeIds.size());
+
+                    for (UUID nodeId : nodeIds)
+                        writer.writeUuid(nodeId);
+                }
+
+                PlatformServiceMethod ann = method.getDeclaredAnnotation(PlatformServiceMethod.class);
+
+                writer.writeString(ann != null ? ann.value() : method.getName());
+
+                Class<?>[] paramTypes = method.getParameterTypes();
+                int[] paramTypeIds = new int[paramTypes.length];

Review comment:
       Unnecessary allocation, we can just iterate and write ids as we go

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/service/ClientServiceInvokeRequest.java
##########
@@ -0,0 +1,304 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.platform.client.service;
+
+import java.lang.reflect.Method;
+import java.util.Arrays;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteServices;
+import org.apache.ignite.internal.binary.BinaryContext;
+import org.apache.ignite.internal.binary.BinaryRawReaderEx;
+import org.apache.ignite.internal.binary.BinaryReaderExImpl;
+import org.apache.ignite.internal.cluster.ClusterGroupAdapter;
+import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl;
+import org.apache.ignite.internal.processors.platform.PlatformNativeException;
+import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext;
+import org.apache.ignite.internal.processors.platform.client.ClientObjectResponse;
+import org.apache.ignite.internal.processors.platform.client.ClientRequest;
+import org.apache.ignite.internal.processors.platform.client.ClientResponse;
+import org.apache.ignite.internal.processors.platform.services.PlatformService;
+import org.apache.ignite.internal.processors.platform.services.PlatformServices;
+import org.apache.ignite.internal.processors.service.GridServiceProxy;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.services.Service;
+import org.apache.ignite.services.ServiceDescriptor;
+
+/**
+ * Request to invoke service method.
+ */
+public class ClientServiceInvokeRequest extends ClientRequest {
+    /** Flag keep binary mask. */
+    private static final byte FLAG_KEEP_BINARY_MASK = 0x01;
+
+    /** Methods cache. */
+    private static final Map<MethodDescriptor, Method> methodsCache = new ConcurrentHashMap<>();
+
+    /** Service name. */
+    private final String name;
+
+    /** Flags. */
+    private final byte flags;
+
+    /** Timeout. */
+    private final long timeout;
+
+    /** Nodes. */
+    private final Set<UUID> nodeIds;
+
+    /** Method name. */
+    private final String methodName;
+
+    /** Method parameter type IDs. */
+    private int[] paramTypeIds;
+
+    /** Service arguments. */
+    private final Object[] args;
+
+    /** Objects reader. */
+    private final BinaryRawReaderEx reader;
+
+    /**
+     * Constructor.
+     *
+     * @param reader Reader.
+     */
+    public ClientServiceInvokeRequest(BinaryReaderExImpl reader) {
+        super(reader);
+
+        name = reader.readString();
+
+        flags = reader.readByte();
+
+        timeout = reader.readLong();
+
+        int cnt = reader.readInt();
+
+        nodeIds = U.newHashSet(cnt);
+
+        for (int i = 0; i < cnt; i++)
+            nodeIds.add(reader.readUuid());
+
+        methodName = reader.readString();
+
+        paramTypeIds = reader.readIntArray();
+
+        int argCnt = reader.readInt();
+
+        args = new Object[argCnt];
+
+        // We can't deserialize some types (arrays of user defined types for example) from detached objects.
+        // On the other hand, deserialize should be done as part of process() call (not in constructor) for proper
+        // error handling.
+        // To overcome these issues we store binary reader reference, parse request in constructor (by reading detached
+        // objects), restore arguments starting position in input stream and deserialize arguments from input stream
+        // in process() method.
+        this.reader = reader;

Review comment:
       This is making a big assumption that reader is still valid after this constructor has exited.
   I know that currently this works, but it would be great to avoid this - too fragile.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/service/ClientServiceInvokeRequest.java
##########
@@ -0,0 +1,304 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.platform.client.service;
+
+import java.lang.reflect.Method;
+import java.util.Arrays;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteServices;
+import org.apache.ignite.internal.binary.BinaryContext;
+import org.apache.ignite.internal.binary.BinaryRawReaderEx;
+import org.apache.ignite.internal.binary.BinaryReaderExImpl;
+import org.apache.ignite.internal.cluster.ClusterGroupAdapter;
+import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl;
+import org.apache.ignite.internal.processors.platform.PlatformNativeException;
+import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext;
+import org.apache.ignite.internal.processors.platform.client.ClientObjectResponse;
+import org.apache.ignite.internal.processors.platform.client.ClientRequest;
+import org.apache.ignite.internal.processors.platform.client.ClientResponse;
+import org.apache.ignite.internal.processors.platform.services.PlatformService;
+import org.apache.ignite.internal.processors.platform.services.PlatformServices;
+import org.apache.ignite.internal.processors.service.GridServiceProxy;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.services.Service;
+import org.apache.ignite.services.ServiceDescriptor;
+
+/**
+ * Request to invoke service method.
+ */
+public class ClientServiceInvokeRequest extends ClientRequest {
+    /** Flag keep binary mask. */
+    private static final byte FLAG_KEEP_BINARY_MASK = 0x01;
+
+    /** Methods cache. */
+    private static final Map<MethodDescriptor, Method> methodsCache = new ConcurrentHashMap<>();
+
+    /** Service name. */
+    private final String name;
+
+    /** Flags. */
+    private final byte flags;
+
+    /** Timeout. */
+    private final long timeout;
+
+    /** Nodes. */
+    private final Set<UUID> nodeIds;
+
+    /** Method name. */
+    private final String methodName;
+
+    /** Method parameter type IDs. */
+    private int[] paramTypeIds;
+
+    /** Service arguments. */
+    private final Object[] args;
+
+    /** Objects reader. */
+    private final BinaryRawReaderEx reader;
+
+    /**
+     * Constructor.
+     *
+     * @param reader Reader.
+     */
+    public ClientServiceInvokeRequest(BinaryReaderExImpl reader) {
+        super(reader);
+
+        name = reader.readString();
+
+        flags = reader.readByte();
+
+        timeout = reader.readLong();
+
+        int cnt = reader.readInt();
+
+        nodeIds = U.newHashSet(cnt);
+
+        for (int i = 0; i < cnt; i++)
+            nodeIds.add(reader.readUuid());
+
+        methodName = reader.readString();
+
+        paramTypeIds = reader.readIntArray();
+
+        int argCnt = reader.readInt();
+
+        args = new Object[argCnt];
+
+        // We can't deserialize some types (arrays of user defined types for example) from detached objects.
+        // On the other hand, deserialize should be done as part of process() call (not in constructor) for proper
+        // error handling.
+        // To overcome these issues we store binary reader reference, parse request in constructor (by reading detached
+        // objects), restore arguments starting position in input stream and deserialize arguments from input stream
+        // in process() method.
+        this.reader = reader;
+
+        int argsStartPos = reader.in().position();
+
+        for (int i = 0; i < argCnt; i++)
+            args[i] = reader.readObjectDetached();
+
+        reader.in().position(argsStartPos);
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClientResponse process(ClientConnectionContext ctx) {
+        if (F.isEmpty(name))
+            throw new IgniteException("Service name can't be empty");
+
+        if (F.isEmpty(methodName))
+            throw new IgniteException("Method name can't be empty");
+
+        ServiceDescriptor desc = null;
+
+        for (ServiceDescriptor desc0 : ctx.kernalContext().service().serviceDescriptors()) {
+            if (name.equals(desc0.name())) {
+                desc = desc0;
+
+                break;
+            }
+        }
+
+        if (desc == null)
+            throw new IgniteException("Service not found: " + name);

Review comment:
       Please extract a method for better readability

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/service/ClientServiceInvokeRequest.java
##########
@@ -0,0 +1,304 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.platform.client.service;
+
+import java.lang.reflect.Method;
+import java.util.Arrays;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteServices;
+import org.apache.ignite.internal.binary.BinaryContext;
+import org.apache.ignite.internal.binary.BinaryRawReaderEx;
+import org.apache.ignite.internal.binary.BinaryReaderExImpl;
+import org.apache.ignite.internal.cluster.ClusterGroupAdapter;
+import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl;
+import org.apache.ignite.internal.processors.platform.PlatformNativeException;
+import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext;
+import org.apache.ignite.internal.processors.platform.client.ClientObjectResponse;
+import org.apache.ignite.internal.processors.platform.client.ClientRequest;
+import org.apache.ignite.internal.processors.platform.client.ClientResponse;
+import org.apache.ignite.internal.processors.platform.services.PlatformService;
+import org.apache.ignite.internal.processors.platform.services.PlatformServices;
+import org.apache.ignite.internal.processors.service.GridServiceProxy;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.services.Service;
+import org.apache.ignite.services.ServiceDescriptor;
+
+/**
+ * Request to invoke service method.
+ */
+public class ClientServiceInvokeRequest extends ClientRequest {
+    /** Flag keep binary mask. */
+    private static final byte FLAG_KEEP_BINARY_MASK = 0x01;
+
+    /** Methods cache. */
+    private static final Map<MethodDescriptor, Method> methodsCache = new ConcurrentHashMap<>();
+
+    /** Service name. */
+    private final String name;
+
+    /** Flags. */
+    private final byte flags;
+
+    /** Timeout. */
+    private final long timeout;
+
+    /** Nodes. */
+    private final Set<UUID> nodeIds;
+
+    /** Method name. */
+    private final String methodName;
+
+    /** Method parameter type IDs. */
+    private int[] paramTypeIds;
+
+    /** Service arguments. */
+    private final Object[] args;
+
+    /** Objects reader. */
+    private final BinaryRawReaderEx reader;
+
+    /**
+     * Constructor.
+     *
+     * @param reader Reader.
+     */
+    public ClientServiceInvokeRequest(BinaryReaderExImpl reader) {
+        super(reader);
+
+        name = reader.readString();
+
+        flags = reader.readByte();
+
+        timeout = reader.readLong();
+
+        int cnt = reader.readInt();
+
+        nodeIds = U.newHashSet(cnt);
+
+        for (int i = 0; i < cnt; i++)
+            nodeIds.add(reader.readUuid());
+
+        methodName = reader.readString();
+
+        paramTypeIds = reader.readIntArray();
+
+        int argCnt = reader.readInt();
+
+        args = new Object[argCnt];
+
+        // We can't deserialize some types (arrays of user defined types for example) from detached objects.
+        // On the other hand, deserialize should be done as part of process() call (not in constructor) for proper
+        // error handling.
+        // To overcome these issues we store binary reader reference, parse request in constructor (by reading detached
+        // objects), restore arguments starting position in input stream and deserialize arguments from input stream
+        // in process() method.
+        this.reader = reader;
+
+        int argsStartPos = reader.in().position();
+
+        for (int i = 0; i < argCnt; i++)
+            args[i] = reader.readObjectDetached();
+
+        reader.in().position(argsStartPos);
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClientResponse process(ClientConnectionContext ctx) {
+        if (F.isEmpty(name))
+            throw new IgniteException("Service name can't be empty");
+
+        if (F.isEmpty(methodName))
+            throw new IgniteException("Method name can't be empty");
+
+        ServiceDescriptor desc = null;
+
+        for (ServiceDescriptor desc0 : ctx.kernalContext().service().serviceDescriptors()) {
+            if (name.equals(desc0.name())) {
+                desc = desc0;
+
+                break;
+            }
+        }
+
+        if (desc == null)
+            throw new IgniteException("Service not found: " + name);
+
+        Class<?> svcCls = desc.serviceClass();
+
+        ClusterGroupAdapter grp = ctx.kernalContext().cluster().get();
+
+        if (ctx.securityContext() != null)
+            grp = (ClusterGroupAdapter)grp.forSubjectId(ctx.securityContext().subject().id());
+
+        grp = (ClusterGroupAdapter)(nodeIds.isEmpty() ? grp.forServers() : grp.forNodeIds(nodeIds));
+
+        IgniteServices services = grp.services();
+
+        if (!keepBinary() && args.length > 0) {
+            for (int i = 0; i < args.length; i++)
+                args[i] = reader.readObject();
+        }
+
+        try {
+            Object res;
+
+            if (PlatformService.class.isAssignableFrom(svcCls)) {

Review comment:
       Nice, thanks for including platform services :+1: 

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/service/ClientServiceInvokeRequest.java
##########
@@ -0,0 +1,304 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.platform.client.service;
+
+import java.lang.reflect.Method;
+import java.util.Arrays;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteServices;
+import org.apache.ignite.internal.binary.BinaryContext;
+import org.apache.ignite.internal.binary.BinaryRawReaderEx;
+import org.apache.ignite.internal.binary.BinaryReaderExImpl;
+import org.apache.ignite.internal.cluster.ClusterGroupAdapter;
+import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl;
+import org.apache.ignite.internal.processors.platform.PlatformNativeException;
+import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext;
+import org.apache.ignite.internal.processors.platform.client.ClientObjectResponse;
+import org.apache.ignite.internal.processors.platform.client.ClientRequest;
+import org.apache.ignite.internal.processors.platform.client.ClientResponse;
+import org.apache.ignite.internal.processors.platform.services.PlatformService;
+import org.apache.ignite.internal.processors.platform.services.PlatformServices;
+import org.apache.ignite.internal.processors.service.GridServiceProxy;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.services.Service;
+import org.apache.ignite.services.ServiceDescriptor;
+
+/**
+ * Request to invoke service method.
+ */
+public class ClientServiceInvokeRequest extends ClientRequest {
+    /** Flag keep binary mask. */
+    private static final byte FLAG_KEEP_BINARY_MASK = 0x01;
+
+    /** Methods cache. */
+    private static final Map<MethodDescriptor, Method> methodsCache = new ConcurrentHashMap<>();
+
+    /** Service name. */
+    private final String name;
+
+    /** Flags. */
+    private final byte flags;
+
+    /** Timeout. */
+    private final long timeout;
+
+    /** Nodes. */
+    private final Set<UUID> nodeIds;
+
+    /** Method name. */
+    private final String methodName;
+
+    /** Method parameter type IDs. */
+    private int[] paramTypeIds;

Review comment:
       `final`

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientServicesImpl.java
##########
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.client.thin;
+
+import java.lang.reflect.InvocationHandler;
+import java.lang.reflect.Method;
+import java.lang.reflect.Proxy;
+import java.util.Collection;
+import java.util.UUID;
+import org.apache.ignite.client.ClientClusterGroup;
+import org.apache.ignite.client.ClientException;
+import org.apache.ignite.client.ClientServices;
+import org.apache.ignite.internal.binary.BinaryRawWriterEx;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.A;
+import org.apache.ignite.platform.PlatformServiceMethod;
+
+/**
+ * Implementation of {@link ClientServices}.
+ */
+class ClientServicesImpl implements ClientServices {
+    /** Channel. */
+    private final ReliableChannel ch;
+
+    /** Binary marshaller. */
+    private final ClientBinaryMarshaller marsh;
+
+    /** Utils for serialization/deserialization. */
+    private final ClientUtils utils;
+
+    /** Cluster group. */
+    private final ClientClusterGroupImpl grp;
+
+    /** Constructor. */
+    ClientServicesImpl(ReliableChannel ch, ClientBinaryMarshaller marsh, ClientClusterGroupImpl grp) {
+        this.ch = ch;
+        this.marsh = marsh;
+        this.grp = grp;
+
+        utils = new ClientUtils(marsh);
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClientClusterGroup clusterGroup() {
+        return grp;
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> T serviceProxy(String name, Class<? super T> svcItf) {
+        return serviceProxy(name, svcItf, 0);
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> T serviceProxy(String name, Class<? super T> svcItf, long timeout) {
+        A.notNullOrEmpty(name, "name");
+        A.notNull(svcItf, "svcItf");
+
+        Collection<UUID> nodeIds = grp.nodeIds();

Review comment:
       This locks the resulting proxy to the cluster group state at the given moment. I'm not sure this is right.
   Should we pass the cluster group to the proxy instead and call nodeIds on every invocation?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/service/ClientServiceInvokeRequest.java
##########
@@ -0,0 +1,304 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.platform.client.service;
+
+import java.lang.reflect.Method;
+import java.util.Arrays;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteServices;
+import org.apache.ignite.internal.binary.BinaryContext;
+import org.apache.ignite.internal.binary.BinaryRawReaderEx;
+import org.apache.ignite.internal.binary.BinaryReaderExImpl;
+import org.apache.ignite.internal.cluster.ClusterGroupAdapter;
+import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl;
+import org.apache.ignite.internal.processors.platform.PlatformNativeException;
+import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext;
+import org.apache.ignite.internal.processors.platform.client.ClientObjectResponse;
+import org.apache.ignite.internal.processors.platform.client.ClientRequest;
+import org.apache.ignite.internal.processors.platform.client.ClientResponse;
+import org.apache.ignite.internal.processors.platform.services.PlatformService;
+import org.apache.ignite.internal.processors.platform.services.PlatformServices;
+import org.apache.ignite.internal.processors.service.GridServiceProxy;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.services.Service;
+import org.apache.ignite.services.ServiceDescriptor;
+
+/**
+ * Request to invoke service method.
+ */
+public class ClientServiceInvokeRequest extends ClientRequest {
+    /** Flag keep binary mask. */
+    private static final byte FLAG_KEEP_BINARY_MASK = 0x01;
+
+    /** Methods cache. */
+    private static final Map<MethodDescriptor, Method> methodsCache = new ConcurrentHashMap<>();
+
+    /** Service name. */
+    private final String name;
+
+    /** Flags. */
+    private final byte flags;
+
+    /** Timeout. */
+    private final long timeout;
+
+    /** Nodes. */
+    private final Set<UUID> nodeIds;
+
+    /** Method name. */
+    private final String methodName;
+
+    /** Method parameter type IDs. */
+    private int[] paramTypeIds;
+
+    /** Service arguments. */
+    private final Object[] args;
+
+    /** Objects reader. */
+    private final BinaryRawReaderEx reader;
+
+    /**
+     * Constructor.
+     *
+     * @param reader Reader.
+     */
+    public ClientServiceInvokeRequest(BinaryReaderExImpl reader) {
+        super(reader);
+
+        name = reader.readString();
+
+        flags = reader.readByte();
+
+        timeout = reader.readLong();
+
+        int cnt = reader.readInt();
+
+        nodeIds = U.newHashSet(cnt);
+
+        for (int i = 0; i < cnt; i++)
+            nodeIds.add(reader.readUuid());
+
+        methodName = reader.readString();
+
+        paramTypeIds = reader.readIntArray();
+
+        int argCnt = reader.readInt();
+
+        args = new Object[argCnt];
+
+        // We can't deserialize some types (arrays of user defined types for example) from detached objects.
+        // On the other hand, deserialize should be done as part of process() call (not in constructor) for proper
+        // error handling.
+        // To overcome these issues we store binary reader reference, parse request in constructor (by reading detached
+        // objects), restore arguments starting position in input stream and deserialize arguments from input stream
+        // in process() method.
+        this.reader = reader;
+
+        int argsStartPos = reader.in().position();
+
+        for (int i = 0; i < argCnt; i++)
+            args[i] = reader.readObjectDetached();
+
+        reader.in().position(argsStartPos);
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClientResponse process(ClientConnectionContext ctx) {
+        if (F.isEmpty(name))
+            throw new IgniteException("Service name can't be empty");
+
+        if (F.isEmpty(methodName))
+            throw new IgniteException("Method name can't be empty");
+
+        ServiceDescriptor desc = null;
+
+        for (ServiceDescriptor desc0 : ctx.kernalContext().service().serviceDescriptors()) {
+            if (name.equals(desc0.name())) {
+                desc = desc0;
+
+                break;
+            }
+        }
+
+        if (desc == null)
+            throw new IgniteException("Service not found: " + name);
+
+        Class<?> svcCls = desc.serviceClass();
+
+        ClusterGroupAdapter grp = ctx.kernalContext().cluster().get();
+
+        if (ctx.securityContext() != null)
+            grp = (ClusterGroupAdapter)grp.forSubjectId(ctx.securityContext().subject().id());
+
+        grp = (ClusterGroupAdapter)(nodeIds.isEmpty() ? grp.forServers() : grp.forNodeIds(nodeIds));
+
+        IgniteServices services = grp.services();
+
+        if (!keepBinary() && args.length > 0) {
+            for (int i = 0; i < args.length; i++)
+                args[i] = reader.readObject();
+        }
+
+        try {
+            Object res;
+
+            if (PlatformService.class.isAssignableFrom(svcCls)) {
+                PlatformService proxy = services.serviceProxy(name, PlatformService.class, false, timeout);
+
+                res = proxy.invokeMethod(methodName, keepBinary(), !keepBinary(), args);
+            }
+            else {
+                GridServiceProxy<?> proxy = new GridServiceProxy<>(grp, name, Service.class, false, timeout,
+                    ctx.kernalContext());
+
+                Method method = resolveMethod(ctx, svcCls);
+
+                res = proxy.invokeMethod(method, args);
+            }
+
+            return new ClientObjectResponse(requestId(), res);
+        }
+        catch (PlatformNativeException e) {
+            ctx.kernalContext().log(getClass()).error("Failed to invoke platform service", e);
+
+            throw new IgniteException("Failed to invoke platform service, see server logs for details");

Review comment:
       Let's include e as cause.

##########
File path: modules/core/src/test/java/org/apache/ignite/internal/client/thin/ServicesTest.java
##########
@@ -0,0 +1,388 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.client.thin;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.client.ClientClusterGroup;
+import org.apache.ignite.client.ClientException;
+import org.apache.ignite.client.IgniteClient;
+import org.apache.ignite.client.Person;
+import org.apache.ignite.internal.IgniteInternalFuture;
+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;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.junit.Test;
+
+/**
+ * Checks service invocation for thin client.
+ */
+public class ServicesTest extends AbstractThinClientTest {

Review comment:
       Please add a test for exceptions within service methods. Let's make sure that errors messages are adequate.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/service/ClientServiceInvokeRequest.java
##########
@@ -0,0 +1,304 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.platform.client.service;
+
+import java.lang.reflect.Method;
+import java.util.Arrays;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteServices;
+import org.apache.ignite.internal.binary.BinaryContext;
+import org.apache.ignite.internal.binary.BinaryRawReaderEx;
+import org.apache.ignite.internal.binary.BinaryReaderExImpl;
+import org.apache.ignite.internal.cluster.ClusterGroupAdapter;
+import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl;
+import org.apache.ignite.internal.processors.platform.PlatformNativeException;
+import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext;
+import org.apache.ignite.internal.processors.platform.client.ClientObjectResponse;
+import org.apache.ignite.internal.processors.platform.client.ClientRequest;
+import org.apache.ignite.internal.processors.platform.client.ClientResponse;
+import org.apache.ignite.internal.processors.platform.services.PlatformService;
+import org.apache.ignite.internal.processors.platform.services.PlatformServices;
+import org.apache.ignite.internal.processors.service.GridServiceProxy;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.services.Service;
+import org.apache.ignite.services.ServiceDescriptor;
+
+/**
+ * Request to invoke service method.
+ */
+public class ClientServiceInvokeRequest extends ClientRequest {
+    /** Flag keep binary mask. */
+    private static final byte FLAG_KEEP_BINARY_MASK = 0x01;
+
+    /** Methods cache. */
+    private static final Map<MethodDescriptor, Method> methodsCache = new ConcurrentHashMap<>();
+
+    /** Service name. */
+    private final String name;
+
+    /** Flags. */
+    private final byte flags;
+
+    /** Timeout. */
+    private final long timeout;
+
+    /** Nodes. */
+    private final Set<UUID> nodeIds;
+
+    /** Method name. */
+    private final String methodName;
+
+    /** Method parameter type IDs. */
+    private int[] paramTypeIds;
+
+    /** Service arguments. */
+    private final Object[] args;
+
+    /** Objects reader. */
+    private final BinaryRawReaderEx reader;
+
+    /**
+     * Constructor.
+     *
+     * @param reader Reader.
+     */
+    public ClientServiceInvokeRequest(BinaryReaderExImpl reader) {
+        super(reader);
+
+        name = reader.readString();
+
+        flags = reader.readByte();
+
+        timeout = reader.readLong();
+
+        int cnt = reader.readInt();
+
+        nodeIds = U.newHashSet(cnt);
+
+        for (int i = 0; i < cnt; i++)
+            nodeIds.add(reader.readUuid());
+
+        methodName = reader.readString();
+
+        paramTypeIds = reader.readIntArray();
+
+        int argCnt = reader.readInt();
+
+        args = new Object[argCnt];
+
+        // We can't deserialize some types (arrays of user defined types for example) from detached objects.
+        // On the other hand, deserialize should be done as part of process() call (not in constructor) for proper
+        // error handling.
+        // To overcome these issues we store binary reader reference, parse request in constructor (by reading detached
+        // objects), restore arguments starting position in input stream and deserialize arguments from input stream
+        // in process() method.
+        this.reader = reader;
+
+        int argsStartPos = reader.in().position();
+
+        for (int i = 0; i < argCnt; i++)
+            args[i] = reader.readObjectDetached();
+
+        reader.in().position(argsStartPos);
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClientResponse process(ClientConnectionContext ctx) {
+        if (F.isEmpty(name))
+            throw new IgniteException("Service name can't be empty");
+
+        if (F.isEmpty(methodName))
+            throw new IgniteException("Method name can't be empty");
+
+        ServiceDescriptor desc = null;
+
+        for (ServiceDescriptor desc0 : ctx.kernalContext().service().serviceDescriptors()) {
+            if (name.equals(desc0.name())) {
+                desc = desc0;
+
+                break;
+            }
+        }
+
+        if (desc == null)
+            throw new IgniteException("Service not found: " + name);
+
+        Class<?> svcCls = desc.serviceClass();
+
+        ClusterGroupAdapter grp = ctx.kernalContext().cluster().get();
+
+        if (ctx.securityContext() != null)
+            grp = (ClusterGroupAdapter)grp.forSubjectId(ctx.securityContext().subject().id());
+
+        grp = (ClusterGroupAdapter)(nodeIds.isEmpty() ? grp.forServers() : grp.forNodeIds(nodeIds));
+
+        IgniteServices services = grp.services();
+
+        if (!keepBinary() && args.length > 0) {
+            for (int i = 0; i < args.length; i++)
+                args[i] = reader.readObject();
+        }
+
+        try {
+            Object res;
+
+            if (PlatformService.class.isAssignableFrom(svcCls)) {
+                PlatformService proxy = services.serviceProxy(name, PlatformService.class, false, timeout);
+
+                res = proxy.invokeMethod(methodName, keepBinary(), !keepBinary(), args);
+            }
+            else {
+                GridServiceProxy<?> proxy = new GridServiceProxy<>(grp, name, Service.class, false, timeout,
+                    ctx.kernalContext());
+
+                Method method = resolveMethod(ctx, svcCls);
+
+                res = proxy.invokeMethod(method, args);
+            }
+
+            return new ClientObjectResponse(requestId(), res);
+        }
+        catch (PlatformNativeException e) {
+            ctx.kernalContext().log(getClass()).error("Failed to invoke platform service", e);
+
+            throw new IgniteException("Failed to invoke platform service, see server logs for details");
+        }
+        catch (Throwable e) {
+            throw new IgniteException(e);
+        }
+    }
+
+    /**
+     * Keep binary flag.
+     */
+    private boolean keepBinary() {
+        return (flags & FLAG_KEEP_BINARY_MASK) != 0;
+    }
+
+    /**
+     * Resolve method by method name and parameter types or parameter values.
+     */
+    private Method resolveMethod(ClientConnectionContext ctx, Class<?> cls) throws ReflectiveOperationException {
+        if (paramTypeIds != null) {
+            MethodDescriptor desc = new MethodDescriptor(cls, methodName, paramTypeIds);
+
+            Method method = methodsCache.get(desc);
+
+            if (method != null)
+                return method;
+
+            BinaryContext bctx = ((CacheObjectBinaryProcessorImpl)ctx.kernalContext().cacheObjects()).binaryContext();
+
+            for (Method method0 : cls.getMethods()) {
+                if (methodName.equals(method0.getName())) {

Review comment:
       What if there are multiple methods with the same name?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org