You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by yz...@apache.org on 2017/10/13 17:42:58 UTC

[13/50] [abbrv] ignite git commit: IGNITE-6250 .NET: Thin client: Basic exception handling

IGNITE-6250 .NET: Thin client: Basic exception handling

This closes #2698


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

Branch: refs/heads/ignite-2.1.5-p1
Commit: bd60ab1fa6aaf433849e8a052af41a2a333c7904
Parents: 1a64615
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Wed Sep 20 16:49:49 2017 +0300
Committer: devozerov <pp...@gmail.com>
Committed: Wed Sep 20 18:12:58 2017 +0300

----------------------------------------------------------------------
 .../odbc/ClientListenerNioListener.java         |   2 +-
 .../odbc/ClientListenerRequestHandler.java      |   3 +-
 .../odbc/jdbc/JdbcRequestHandler.java           |   2 +-
 .../odbc/odbc/OdbcRequestHandler.java           |   2 +-
 .../client/ClientConnectionContext.java         |   4 +-
 .../platform/client/ClientMessageParser.java    |  20 ++-
 .../platform/client/ClientRawRequest.java       |  45 +++++++
 .../platform/client/ClientRequest.java          |   9 ++
 .../platform/client/ClientRequestHandler.java   |  17 +--
 .../platform/client/ClientResponse.java         |  33 ++++-
 .../platform/client/ClientStatus.java           |  45 +++++++
 .../platform/client/IgniteClientException.java  |  65 ++++++++++
 .../client/cache/ClientCacheRequest.java        |  11 +-
 .../Client/Cache/CacheTest.cs                   |  20 +++
 .../Client/Cache/ScanQueryTest.cs               |  34 ++++-
 .../Client/ClientConnectionTest.cs              |   5 +-
 .../Client/RawSocketTest.cs                     | 130 +++++++++++++------
 .../Apache.Ignite.Core.csproj                   |   2 +
 .../Client/IgniteClientException.cs             | 108 +++++++++++++++
 .../Impl/Client/Cache/CacheClient.cs            |  23 +++-
 .../Impl/Client/ClientSocket.cs                 |  21 +--
 .../Impl/Client/ClientStatus.cs                 |  31 +++++
 22 files changed, 546 insertions(+), 86 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/bd60ab1f/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerNioListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerNioListener.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerNioListener.java
index f0f4903..faecab3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerNioListener.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerNioListener.java
@@ -154,7 +154,7 @@ public class ClientListenerNioListener extends GridNioServerListenerAdapter<byte
         catch (Exception e) {
             log.error("Failed to process client request [req=" + req + ']', e);
 
-            ses.send(parser.encode(handler.handleException(e)));
+            ses.send(parser.encode(handler.handleException(e, req)));
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/bd60ab1f/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerRequestHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerRequestHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerRequestHandler.java
index e28d0d9..cebde08 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerRequestHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerRequestHandler.java
@@ -35,9 +35,10 @@ public interface ClientListenerRequestHandler {
      * Handle exception.
      *
      * @param e Exception.
+     * @param req Request.
      * @return Error response.
      */
-    public ClientListenerResponse handleException(Exception e);
+    public ClientListenerResponse handleException(Exception e, ClientListenerRequest req);
 
     /**
      * Write successful handshake response.

http://git-wip-us.apache.org/repos/asf/ignite/blob/bd60ab1f/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java
index 2143fb6f..ea25b11 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java
@@ -188,7 +188,7 @@ public class JdbcRequestHandler implements ClientListenerRequestHandler {
     }
 
     /** {@inheritDoc} */
-    @Override public ClientListenerResponse handleException(Exception e) {
+    @Override public ClientListenerResponse handleException(Exception e, ClientListenerRequest req) {
         return exceptionToResult(e);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/bd60ab1f/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandler.java
index cca991d..fd31f39 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandler.java
@@ -160,7 +160,7 @@ public class OdbcRequestHandler implements ClientListenerRequestHandler {
     }
 
     /** {@inheritDoc} */
-    @Override public ClientListenerResponse handleException(Exception e) {
+    @Override public ClientListenerResponse handleException(Exception e, ClientListenerRequest req) {
         return new OdbcResponse(ClientListenerResponse.STATUS_FAILED, e.toString());
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/bd60ab1f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientConnectionContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientConnectionContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientConnectionContext.java
index 7bab76f..2d81f35 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientConnectionContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientConnectionContext.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.processors.platform.client;
 
-import org.apache.ignite.IgniteException;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.binary.BinaryReaderExImpl;
 import org.apache.ignite.internal.processors.odbc.ClientListenerConnectionContext;
@@ -123,7 +122,8 @@ public class ClientConnectionContext implements ClientListenerConnectionContext
         long curCnt0 = curCnt.get();
 
         if (curCnt0 >= maxCursors) {
-            throw new IgniteException("Too many open cursors (either close other open cursors or increase the " +
+            throw new IgniteClientException(ClientStatus.TOO_MANY_CURSORS,
+                "Too many open cursors (either close other open cursors or increase the " +
                 "limit through ClientConnectorConfiguration.maxOpenCursorsPerConnection) [maximum=" + maxCursors +
                 ", current=" + curCnt0 + ']');
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/bd60ab1f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientMessageParser.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientMessageParser.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientMessageParser.java
index 219afdc..84d3eee 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientMessageParser.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientMessageParser.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.processors.platform.client;
 
-import org.apache.ignite.IgniteException;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.binary.BinaryRawReaderEx;
 import org.apache.ignite.internal.binary.BinaryRawWriterEx;
@@ -29,10 +28,10 @@ import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProce
 import org.apache.ignite.internal.processors.odbc.ClientListenerMessageParser;
 import org.apache.ignite.internal.processors.odbc.ClientListenerRequest;
 import org.apache.ignite.internal.processors.odbc.ClientListenerResponse;
-import org.apache.ignite.internal.processors.platform.client.binary.ClientBinaryTypeNameGetRequest;
 import org.apache.ignite.internal.processors.platform.client.binary.ClientBinaryTypeGetRequest;
-import org.apache.ignite.internal.processors.platform.client.binary.ClientBinaryTypePutRequest;
+import org.apache.ignite.internal.processors.platform.client.binary.ClientBinaryTypeNameGetRequest;
 import org.apache.ignite.internal.processors.platform.client.binary.ClientBinaryTypeNamePutRequest;
+import org.apache.ignite.internal.processors.platform.client.binary.ClientBinaryTypePutRequest;
 import org.apache.ignite.internal.processors.platform.client.cache.ClientCacheGetRequest;
 import org.apache.ignite.internal.processors.platform.client.cache.ClientCachePutRequest;
 import org.apache.ignite.internal.processors.platform.client.cache.ClientCacheScanQueryNextPageRequest;
@@ -91,6 +90,16 @@ public class ClientMessageParser implements ClientListenerMessageParser {
         BinaryInputStream inStream = new BinaryHeapInputStream(msg);
         BinaryRawReaderEx reader = marsh.reader(inStream);
 
+        return decode(reader);
+    }
+
+    /**
+     * Decodes the request.
+     *
+     * @param reader Reader.
+     * @return Request.
+     */
+    public ClientListenerRequest decode(BinaryRawReaderEx reader) {
         short opCode = reader.readShort();
 
         switch (opCode) {
@@ -122,11 +131,14 @@ public class ClientMessageParser implements ClientListenerMessageParser {
                 return new ClientResourceCloseRequest(reader);
         }
 
-        throw new IgniteException("Invalid operation: " + opCode);
+        return new ClientRawRequest(reader.readLong(), ClientStatus.INVALID_OP_CODE,
+            "Invalid request op code: " + opCode);
     }
 
     /** {@inheritDoc} */
     @Override public byte[] encode(ClientListenerResponse resp) {
+        assert resp != null;
+
         BinaryHeapOutputStream outStream = new BinaryHeapOutputStream(32);
 
         BinaryRawWriterEx writer = marsh.writer(outStream);

http://git-wip-us.apache.org/repos/asf/ignite/blob/bd60ab1f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientRawRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientRawRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientRawRequest.java
new file mode 100644
index 0000000..e6c12de
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientRawRequest.java
@@ -0,0 +1,45 @@
+/*
+ * 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;
+
+/**
+ * Raw request, produces a ClientResponse directly.
+ */
+public class ClientRawRequest extends ClientRequest {
+    /** Status code. */
+    private final int status;
+
+    /** Message. */
+    private final String msg;
+
+    /**
+     * Initializes a new instance of ClientRawRequest class.
+     * @param reqId Request id.
+     * @param msg Message.
+     */
+    public ClientRawRequest(long reqId, int status, String msg) {
+        super(reqId);
+        this.status = status;
+        this.msg = msg;
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClientResponse process(ClientConnectionContext ctx) {
+        return new ClientResponse(requestId(), status, msg);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/bd60ab1f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientRequest.java
index 788d2e0..76823b5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientRequest.java
@@ -36,6 +36,15 @@ public class ClientRequest implements ClientListenerRequest {
         reqId = reader.readLong();
     }
 
+    /**
+     * Constructor.
+     *
+     * @param reqId Request id.
+     */
+    public ClientRequest(long reqId) {
+        this.reqId = reqId;
+    }
+
     /** {@inheritDoc} */
     @Override public long requestId() {
         return reqId;

http://git-wip-us.apache.org/repos/asf/ignite/blob/bd60ab1f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientRequestHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientRequestHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientRequestHandler.java
index 499fb05..3f6c082 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientRequestHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientRequestHandler.java
@@ -42,17 +42,18 @@ public class ClientRequestHandler implements ClientListenerRequestHandler {
 
     /** {@inheritDoc} */
     @Override public ClientListenerResponse handle(ClientListenerRequest req) {
-        try {
-            return ((ClientRequest)req).process(ctx);
-        }
-        catch (Throwable e) {
-            return new ClientResponse(req.requestId(), e.getMessage());
-        }
+        return ((ClientRequest) req).process(ctx);
     }
 
     /** {@inheritDoc} */
-    @Override public ClientListenerResponse handleException(Exception e) {
-        return null;
+    @Override public ClientListenerResponse handleException(Exception e, ClientListenerRequest req) {
+        assert req != null;
+        assert e != null;
+
+        int status = e instanceof IgniteClientException ?
+            ((IgniteClientException)e).statusCode() : ClientStatus.FAILED;
+
+        return new ClientResponse(req.requestId(), status, e.getMessage());
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/bd60ab1f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientResponse.java
index a1bc4b6..6bffac3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientResponse.java
@@ -33,7 +33,7 @@ public class ClientResponse extends ClientListenerResponse {
      * @param reqId Request id.
      */
     public ClientResponse(long reqId) {
-        super(STATUS_SUCCESS, null);
+        super(ClientStatus.SUCCESS, null);
 
         this.reqId = reqId;
     }
@@ -42,9 +42,23 @@ public class ClientResponse extends ClientListenerResponse {
      * Constructor.
      *
      * @param reqId Request id.
+     * @param err Error message.
      */
     public ClientResponse(long reqId, String err) {
-        super(STATUS_FAILED, err);
+        super(ClientStatus.FAILED, err);
+
+        this.reqId = reqId;
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param reqId Request id.
+     * @param status Status code.
+     * @param err Error message.
+     */
+    public ClientResponse(long reqId, int status, String err) {
+        super(status, err);
 
         this.reqId = reqId;
     }
@@ -54,12 +68,19 @@ public class ClientResponse extends ClientListenerResponse {
      */
     public void encode(BinaryRawWriterEx writer) {
         writer.writeLong(reqId);
+        writer.writeInt(status());
 
-        if (status() == STATUS_SUCCESS) {
-            writer.writeBoolean(true);
-        } else {
-            writer.writeBoolean(false);
+        if (status() != ClientStatus.SUCCESS) {
             writer.writeString(error());
         }
     }
+
+    /**
+     * Gets the request id.
+     *
+     * @return Request id.
+     */
+    public long requestId() {
+        return reqId;
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/bd60ab1f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientStatus.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientStatus.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientStatus.java
new file mode 100644
index 0000000..0f374ff
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientStatus.java
@@ -0,0 +1,45 @@
+/*
+ * 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;
+
+/**
+ * Client status codes.
+ */
+public final class ClientStatus {
+    /**
+     * No-op constructor to prevent instantiation.
+     */
+    private ClientStatus (){
+        // No-op.
+    }
+
+    /** Command succeeded. */
+    public static final int SUCCESS = 0;
+
+    /** Command failed. */
+    public static final int FAILED = 1;
+
+    /** Invalid op code. */
+    public static final int INVALID_OP_CODE = 2;
+
+    /** Cache does not exist. */
+    public static final int CACHE_DOES_NOT_EXIST = 1000;
+
+    /** Too many cursors. */
+    public static final int TOO_MANY_CURSORS = 1010;
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/bd60ab1f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/IgniteClientException.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/IgniteClientException.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/IgniteClientException.java
new file mode 100644
index 0000000..9bcabab
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/IgniteClientException.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.platform.client;
+
+import org.apache.ignite.IgniteException;
+
+import java.sql.SQLException;
+
+/**
+ * Client exception.
+ */
+public class IgniteClientException extends IgniteException {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Code to return as {@link SQLException#vendorCode} */
+    private final int statusCode;
+
+    /**
+     * Constructor.
+     *
+     * @param statusCode Status code (see {@link ClientStatus}).
+     * @param msg Message.
+     */
+    public IgniteClientException(int statusCode, String msg) {
+        this(statusCode, msg, null);
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param statusCode Status code (see {@link ClientStatus}).
+     * @param msg Message.
+     * @param cause Cause.
+     */
+    public IgniteClientException(int statusCode, String msg, Exception cause) {
+        super(msg, cause);
+
+        this.statusCode = statusCode;
+    }
+
+    /**
+     * Gets the status code.
+     *
+     * @return Status code.
+     */
+    public int statusCode() {
+        return statusCode;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/bd60ab1f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheRequest.java
index fd7cc6c..8f81e94 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheRequest.java
@@ -19,8 +19,11 @@ package org.apache.ignite.internal.processors.platform.client.cache;
 
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.binary.BinaryRawReader;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext;
+import org.apache.ignite.internal.processors.platform.client.IgniteClientException;
 import org.apache.ignite.internal.processors.platform.client.ClientRequest;
+import org.apache.ignite.internal.processors.platform.client.ClientStatus;
 
 /**
  * Cache get request.
@@ -74,7 +77,13 @@ class ClientCacheRequest extends ClientRequest {
      * @return Cache.
      */
     protected IgniteCache rawCache(ClientConnectionContext ctx) {
-        String cacheName = ctx.kernalContext().cache().context().cacheContext(cacheId).cache().name();
+        GridCacheContext<Object, Object> cacheCtx = ctx.kernalContext().cache().context().cacheContext(cacheId);
+
+        if (cacheCtx == null)
+            throw new IgniteClientException(ClientStatus.CACHE_DOES_NOT_EXIST, "Cache does not exist [cacheId= " +
+                cacheId + "]", null);
+
+        String cacheName = cacheCtx.cache().name();
 
         return ctx.kernalContext().grid().cache(cacheName);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/bd60ab1f/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/CacheTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/CacheTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/CacheTest.cs
index 6cc0662..4293c5b 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/CacheTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/CacheTest.cs
@@ -25,6 +25,7 @@ namespace Apache.Ignite.Core.Tests.Client.Cache
     using System.Threading;
     using Apache.Ignite.Core.Binary;
     using Apache.Ignite.Core.Client;
+    using Apache.Ignite.Core.Impl.Client;
     using NUnit.Framework;
 
     /// <summary>
@@ -151,5 +152,24 @@ namespace Apache.Ignite.Core.Tests.Client.Cache
 
             clients.ToList().ForEach(x => x.Value.Dispose());
         }
+
+        /// <summary>
+        /// Tests the cache exceptions.
+        /// </summary>
+        [Test]
+        public void TestExceptions()
+        {
+            using (var client = GetClient())
+            {
+                // Getting the cache instance does not throw.
+                var cache = client.GetCache<int, int>("foobar");
+
+                // Accessing non-existent cache throws.
+                var ex = Assert.Throws<IgniteClientException>(() => cache.Put(1, 1));
+
+                Assert.AreEqual("Cache doesn't exist: foobar", ex.Message);
+                Assert.AreEqual((int) ClientStatus.CacheDoesNotExist, ex.ErrorCode);
+            }
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/bd60ab1f/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/ScanQueryTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/ScanQueryTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/ScanQueryTest.cs
index 7f8b589..17a2b3f 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/ScanQueryTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/ScanQueryTest.cs
@@ -24,8 +24,9 @@ namespace Apache.Ignite.Core.Tests.Client.Cache
     using System.Linq;
     using Apache.Ignite.Core.Cache;
     using Apache.Ignite.Core.Cache.Query;
-    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Client;
     using Apache.Ignite.Core.Configuration;
+    using Apache.Ignite.Core.Impl.Client;
     using NUnit.Framework;
 
     /// <summary>
@@ -135,6 +136,28 @@ namespace Apache.Ignite.Core.Tests.Client.Cache
         }
 
         /// <summary>
+        /// Tests the exception in filter.
+        /// </summary>
+        [Test]
+        public void TestExceptionInFilter()
+        {
+            GetPersonCache();
+
+            using (var client = GetClient())
+            {
+                var clientCache = client.GetCache<int, Person>(CacheName);
+
+                var qry = new ScanQuery<int, Person>(new PersonFilter(x =>
+                {
+                    throw new ArithmeticException("foo");
+                }));
+
+                var ex = Assert.Throws<IgniteClientException>(() => clientCache.Query(qry).GetAll());
+                Assert.AreEqual("foo", ex.Message);
+            }
+        }
+
+        /// <summary>
         /// Tests multiple cursors with the same client.
         /// </summary>
         [Test]
@@ -154,8 +177,9 @@ namespace Apache.Ignite.Core.Tests.Client.Cache
                 var cur3 = clientCache.Query(qry).GetEnumerator();
 
                 // MaxCursors = 3
-                var ex = Assert.Throws<IgniteException>(() => clientCache.Query(qry));
+                var ex = Assert.Throws<IgniteClientException>(() => clientCache.Query(qry));
                 Assert.AreEqual("Too many open cursors", ex.Message.Substring(0, 21));
+                Assert.AreEqual((int) ClientStatus.TooManyCursors, ex.ErrorCode);
 
                 var count = 0;
 
@@ -177,17 +201,17 @@ namespace Apache.Ignite.Core.Tests.Client.Cache
                 var c2 = clientCache.Query(qry);
                 var c3 = clientCache.Query(qry);
 
-                Assert.Throws<IgniteException>(() => clientCache.Query(qry));
+                Assert.Throws<IgniteClientException>(() => clientCache.Query(qry));
 
                 // Close one of the cursors.
                 c1.Dispose();
                 c1 = clientCache.Query(qry);
-                Assert.Throws<IgniteException>(() => clientCache.Query(qry));
+                Assert.Throws<IgniteClientException>(() => clientCache.Query(qry));
 
                 // Close cursor via GetAll.
                 c1.GetAll();
                 c1 = clientCache.Query(qry);
-                Assert.Throws<IgniteException>(() => clientCache.Query(qry));
+                Assert.Throws<IgniteClientException>(() => clientCache.Query(qry));
 
                 c1.Dispose();
                 c2.Dispose();

http://git-wip-us.apache.org/repos/asf/ignite/blob/bd60ab1f/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientConnectionTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientConnectionTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientConnectionTest.cs
index 7dcec55..66aa844 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientConnectionTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientConnectionTest.cs
@@ -22,7 +22,6 @@ namespace Apache.Ignite.Core.Tests.Client
     using System.Net;
     using System.Net.Sockets;
     using Apache.Ignite.Core.Client;
-    using Apache.Ignite.Core.Common;
     using Apache.Ignite.Core.Configuration;
     using Apache.Ignite.Core.Impl.Client;
     using NUnit.Framework;
@@ -119,9 +118,11 @@ namespace Apache.Ignite.Core.Tests.Client
             using (Ignition.Start(TestUtils.GetTestConfiguration()))
             {
                 // ReSharper disable once ObjectCreationAsStatement
-                var ex = Assert.Throws<IgniteException>(() => new ClientSocket(GetClientConfiguration(),
+                var ex = Assert.Throws<IgniteClientException>(() => new ClientSocket(GetClientConfiguration(),
                     new ClientProtocolVersion(-1, -1, -1)));
 
+                Assert.AreEqual((int) ClientStatus.Fail, ex.ErrorCode);
+
                 Assert.AreEqual("Client handhsake failed: 'Unsupported version.'. " +
                                 "Client version: -1.-1.-1. Server version: 1.0.0", ex.Message);
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/bd60ab1f/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/RawSocketTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/RawSocketTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/RawSocketTest.cs
index b34e037..b637e88 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/RawSocketTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/RawSocketTest.cs
@@ -20,17 +20,17 @@ namespace Apache.Ignite.Core.Tests.Client
     using System;
     using System.Net;
     using System.Net.Sockets;
-    using Apache.Ignite.Core.Cache.Configuration;
     using Apache.Ignite.Core.Configuration;
     using Apache.Ignite.Core.Impl;
     using Apache.Ignite.Core.Impl.Binary;
     using Apache.Ignite.Core.Impl.Binary.IO;
+    using Apache.Ignite.Core.Impl.Client;
     using NUnit.Framework;
 
     /// <summary>
     /// Tests the thin client mode with a raw socket.
     /// </summary>
-    public class RawSocketTest
+    public class RawSocketTest : ClientTestBase
     {
         /// <summary>
         /// Tests the socket handshake connection.
@@ -38,59 +38,113 @@ namespace Apache.Ignite.Core.Tests.Client
         [Test]
         public void TestCacheGet()
         {
-            var cfg = new IgniteConfiguration(TestUtils.GetTestConfiguration())
+            var ignite = Ignition.GetIgnite();
+
+            var marsh = ((Ignite) ignite).Marshaller;
+
+            // Create cache.
+            var cache = GetCache<string>();
+            cache[1] = "bar";
+
+            // Connect socket.
+            var sock = GetSocket();
+
+            // Cache get.
+            SendRequest(sock, stream =>
             {
-                ClientConnectorConfiguration = new ClientConnectorConfiguration()
-            };
+                stream.WriteShort(1); // OP_GET
+                stream.WriteLong(1); // Request id.
+                var cacheId = BinaryUtils.GetStringHashCode(cache.Name);
+                stream.WriteInt(cacheId);
+                stream.WriteByte(0); // Flags (withSkipStore, etc)
+
+                var writer = marsh.StartMarshal(stream);
 
-            using (var ignite = Ignition.Start(cfg))
+                writer.WriteObject(1); // Key
+            });
+
+            var msg = ReceiveMessage(sock);
+
+            using (var stream = new BinaryHeapStream(msg))
             {
-                var marsh = ((Ignite) ignite).Marshaller;
+                var reader = marsh.StartUnmarshal(stream);
 
-                // Create cache.
-                var cacheCfg = new CacheConfiguration("foo", new QueryEntity(typeof(int), typeof(string)));
-                var cache = ignite.CreateCache<int, string>(cacheCfg);
-                cache[1] = "bar";
+                var requestId = reader.ReadLong();
+                Assert.AreEqual(1, requestId);
 
-                // Connect socket.
-                var sock = GetSocket(ClientConnectorConfiguration.DefaultPort);
-                Assert.IsTrue(sock.Connected);
+                var status = reader.ReadInt();
+                Assert.AreEqual(0, status); // Success.
 
-                DoHandshake(sock);
+                var res = reader.ReadObject<string>();
+                Assert.AreEqual(cache[1], res);
+            }
+        }
 
-                // Cache get.
-                SendRequest(sock, stream =>
-                {
-                    stream.WriteShort(1);  // OP_GET
-                    stream.WriteLong(1);  // Request id.
-                    var cacheId = BinaryUtils.GetStringHashCode(cache.Name);
-                    stream.WriteInt(cacheId);
-                    stream.WriteByte(0);  // Flags (withSkipStore, etc)
+        /// <summary>
+        /// Tests invalid operation code.
+        /// </summary>
+        [Test]
+        public void TestInvalidOpCode()
+        {
+            // Connect socket.
+            var sock = GetSocket();
 
-                    var writer = marsh.StartMarshal(stream);
+            // Request invalid operation.
+            SendRequest(sock, stream =>
+            {
+                stream.WriteShort(-1);
+                stream.WriteLong(11);  // Request id.
+            });
 
-                    writer.WriteObject(1);  // Key
-                });
+            var msg = ReceiveMessage(sock);
 
-                var msg = ReceiveMessage(sock);
-                
-                using (var stream = new BinaryHeapStream(msg))
-                {
-                    var reader = marsh.StartUnmarshal(stream);
+            using (var stream = new BinaryHeapStream(msg))
+            {
+                var reader = BinaryUtils.Marshaller.StartUnmarshal(stream);
 
-                    var requestId = reader.ReadLong();
-                    Assert.AreEqual(1, requestId);
+                var requestId = reader.ReadLong();
+                Assert.AreEqual(11, requestId);
 
-                    var success = reader.ReadBoolean();
-                    Assert.IsTrue(success);
+                var status = reader.ReadInt();
+                Assert.AreEqual((int) ClientStatus.InvalidOpCode, status);
 
-                    var res = reader.ReadObject<string>();
-                    Assert.AreEqual(cache[1], res);
-                }
+                var err = reader.ReadObject<string>();
+                Assert.AreEqual("Invalid request op code: -1", err);
             }
         }
 
         /// <summary>
+        /// Tests invalid message (can't be parsed).
+        /// </summary>
+        [Test]
+        public void TestInvalidMessage()
+        {
+            // Connect socket.
+            var sock = GetSocket();
+
+            // Request invalid operation.
+            SendRequest(sock, stream => stream.WriteShort(-1));
+
+            var msg = ReceiveMessage(sock);
+
+            Assert.AreEqual(0, msg.Length);
+        }
+
+        /// <summary>
+        /// Gets the socket.
+        /// </summary>
+        /// <returns>Connected socket after handshake.</returns>
+        private static Socket GetSocket()
+        {
+            var sock = GetSocket(ClientConnectorConfiguration.DefaultPort);
+            Assert.IsTrue(sock.Connected);
+
+            DoHandshake(sock);
+
+            return sock;
+        }
+
+        /// <summary>
         /// Does the handshake.
         /// </summary>
         /// <param name="sock">The sock.</param>

http://git-wip-us.apache.org/repos/asf/ignite/blob/bd60ab1f/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
index 75da6c8..28fe741 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
@@ -99,6 +99,7 @@
     <Compile Include="Cache\IMemoryMetrics.cs" />
     <Compile Include="Client\Cache\ICacheClient.cs" />
     <Compile Include="Client\IgniteClientConfiguration.cs" />
+    <Compile Include="Client\IgniteClientException.cs" />
     <Compile Include="Client\IIgniteClient.cs" />
     <Compile Include="Common\ExceptionFactory.cs" />
     <Compile Include="Configuration\Package-Info.cs" />
@@ -109,6 +110,7 @@
     <Compile Include="Impl\Cache\Query\PlatformQueryQursorBase.cs" />
     <Compile Include="Impl\Binary\BinaryProcessorClient.cs" />
     <Compile Include="Impl\Binary\IBinaryProcessor.cs" />
+    <Compile Include="Impl\Client\ClientStatus.cs" />
     <Compile Include="Impl\IIgniteInternal.cs" />
     <Compile Include="Impl\Client\Cache\CacheClient.cs" />
     <Compile Include="Impl\Client\ClientOp.cs" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/bd60ab1f/modules/platforms/dotnet/Apache.Ignite.Core/Client/IgniteClientException.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Client/IgniteClientException.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Client/IgniteClientException.cs
new file mode 100644
index 0000000..a20bec3
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Client/IgniteClientException.cs
@@ -0,0 +1,108 @@
+/*
+ * 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
+{
+    using System;
+    using System.Runtime.Serialization;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Impl.Client;
+
+    /// <summary>
+    /// Ignite thin client exception.
+    /// </summary>
+    [Serializable]
+    public class IgniteClientException : IgniteException
+    {
+        /** Error code field. */
+        private const string ErrorCodeField = "ErrorCode";
+
+        /** Error code. */
+        private readonly int _errorCode = (int) ClientStatus.Fail;
+
+        /// <summary>
+        /// Gets the error code.
+        /// </summary>
+        public int ErrorCode
+        {
+            get { return _errorCode; }
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="IgniteClientException"/> class.
+        /// </summary>
+        public IgniteClientException()
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="IgniteClientException" /> class.
+        /// </summary>
+        /// <param name="message">The message that describes the error.</param>
+        public IgniteClientException(string message) : base(message)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="IgniteClientException" /> class.
+        /// </summary>
+        /// <param name="message">The message.</param>
+        /// <param name="cause">The cause.</param>
+        public IgniteClientException(string message, Exception cause) : base(message, cause)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="IgniteClientException" /> class.
+        /// </summary>
+        /// <param name="message">The message.</param>
+        /// <param name="cause">The cause.</param>
+        /// <param name="errorCode">The error code.</param>
+        public IgniteClientException(string message, Exception cause, int errorCode) : base(message, cause)
+        {
+            _errorCode = errorCode;
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="IgniteClientException"/> class.
+        /// </summary>
+        /// <param name="info">Serialization information.</param>
+        /// <param name="ctx">Streaming context.</param>
+        protected IgniteClientException(SerializationInfo info, StreamingContext ctx) : base(info, ctx)
+        {
+            _errorCode = info.GetInt32(ErrorCodeField);
+        }
+
+        /// <summary>
+        /// When overridden in a derived class, sets the <see cref="SerializationInfo" /> 
+        /// with information about the exception.
+        /// </summary>
+        /// <param name="info">The <see cref="SerializationInfo" /> that holds the serialized object data
+        /// about the exception being thrown.</param>
+        /// <param name="context">The <see cref="StreamingContext" /> that contains contextual information
+        /// about the source or destination.</param>
+        public override void GetObjectData(SerializationInfo info, StreamingContext context)
+        {
+            base.GetObjectData(info, context);
+
+            info.AddValue(ErrorCodeField, _errorCode);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/bd60ab1f/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/Cache/CacheClient.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/Cache/CacheClient.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/Cache/CacheClient.cs
index 974bab3..5492ef8 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/Cache/CacheClient.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/Cache/CacheClient.cs
@@ -79,12 +79,6 @@ namespace Apache.Ignite.Core.Impl.Client.Cache
         }
 
         /** <inheritDoc /> */
-        public IIgniteClient Ignite
-        {
-            get { return _ignite; }
-        }
-
-        /** <inheritDoc /> */
         public TV this[TK key]
         {
             get { return Get(key); }
@@ -142,7 +136,7 @@ namespace Apache.Ignite.Core.Impl.Client.Cache
 
                     _marsh.FinishMarshal(writer);
                 }
-            }, readFunc);
+            }, readFunc, HandleError<T>);
         }
 
         /// <summary>
@@ -199,6 +193,21 @@ namespace Apache.Ignite.Core.Impl.Client.Cache
         }
 
         /// <summary>
+        /// Handles the error.
+        /// </summary>
+        private T HandleError<T>(ClientStatus status, string msg)
+        {
+            switch (status)
+            {
+                case ClientStatus.CacheDoesNotExist:
+                    throw new IgniteClientException("Cache doesn't exist: " + Name, null, (int) status);
+
+                default:
+                    throw new IgniteClientException(msg, null, (int) status);
+            }
+        }
+
+        /// <summary>
         /// Gets the key not found exception.
         /// </summary>
         private static KeyNotFoundException GetKeyNotFoundException()

http://git-wip-us.apache.org/repos/asf/ignite/blob/bd60ab1f/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientSocket.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientSocket.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientSocket.cs
index 1a245a1..4eb53dd 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientSocket.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientSocket.cs
@@ -68,7 +68,7 @@ namespace Apache.Ignite.Core.Impl.Client
         /// Performs a send-receive operation.
         /// </summary>
         public T DoOutInOp<T>(ClientOp opId, Action<IBinaryStream> writeAction,
-            Func<IBinaryStream, T> readFunc)
+            Func<IBinaryStream, T> readFunc, Func<ClientStatus, string, T> errorFunc = null)
         {
             var requestId = Interlocked.Increment(ref _requestId);
 
@@ -88,19 +88,22 @@ namespace Apache.Ignite.Core.Impl.Client
                 var resRequestId = stream.ReadLong();
                 Debug.Assert(requestId == resRequestId);
 
-                if (!stream.ReadBool())
+                var statusCode = (ClientStatus) stream.ReadInt();
+
+                if (statusCode == ClientStatus.Success)
                 {
-                    // Error.
-                    throw new IgniteException(BinaryUtils.Marshaller.StartUnmarshal(stream).ReadString());
+                    return readFunc != null ? readFunc(stream) : default(T);
                 }
 
-                if (readFunc != null)
+                var msg = BinaryUtils.Marshaller.StartUnmarshal(stream).ReadString();
+
+                if (errorFunc != null)
                 {
-                    return readFunc(stream);
+                    return errorFunc(statusCode, msg);
                 }
-            }
 
-            return default(T);
+                throw new IgniteClientException(msg, null, (int) statusCode);
+            }
         }
 
         /// <summary>
@@ -136,7 +139,7 @@ namespace Apache.Ignite.Core.Impl.Client
 
                 var errMsg = BinaryUtils.Marshaller.Unmarshal<string>(stream);
 
-                throw new IgniteException(string.Format(
+                throw new IgniteClientException(string.Format(
                     "Client handhsake failed: '{0}'. Client version: {1}. Server version: {2}",
                     errMsg, version, serverVersion));
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/bd60ab1f/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientStatus.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientStatus.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientStatus.cs
new file mode 100644
index 0000000..a2e66e8
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientStatus.cs
@@ -0,0 +1,31 @@
+/*
+ * 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
+{
+    /// <summary>
+    /// Client status codes.
+    /// </summary>
+    internal enum ClientStatus
+    {
+        Success = 0,
+        Fail = 1,
+        InvalidOpCode = 2,
+        CacheDoesNotExist = 1000,
+        TooManyCursors = 1010
+    }
+}