You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2015/07/22 18:12:01 UTC

[1/3] incubator-ignite git commit: # gg-10561: first impl

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-gg-10561 3194415cd -> 482cfbb8c


# gg-10561: first impl


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

Branch: refs/heads/ignite-gg-10561
Commit: 933940af94eaf8399ca0ca8c3d085aed3fae6dec
Parents: 3194415
Author: ashutak <as...@gridgain.com>
Authored: Wed Jul 22 13:24:52 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Wed Jul 22 13:24:52 2015 +0300

----------------------------------------------------------------------
 .../processors/rest/GridRestProcessor.java      | 105 +++++++++++++++++--
 .../http/jetty/GridJettyRestHandler.java        |   9 ++
 2 files changed, 107 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/933940af/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java
index 2d1d802..a76a119 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java
@@ -89,6 +89,12 @@ public class GridRestProcessor extends GridProcessorAdapter {
         }
     };
 
+    /** Session token to client ID map. */
+    private final ConcurrentMap<UUID, UUID> sesToken2ClientId = new ConcurrentHashMap<>();
+
+    /** Client ID to session token map */
+    private final ConcurrentMap<UUID, UUID> clientId2SesToken = new ConcurrentHashMap<>();
+
     /**
      * @param req Request.
      * @return Future.
@@ -175,6 +181,24 @@ public class GridRestProcessor extends GridProcessorAdapter {
         SecurityContext subjCtx = null;
 
         if (ctx.security().enabled()) {
+            IgniteBiTuple<UUID, byte[]> clientIdAndSesTok;
+
+            try {
+                clientIdAndSesTok = clientIdAndSessionToken(req);
+            }
+            catch (IgniteCheckedException e) {
+                GridRestResponse res = new GridRestResponse(STATUS_FAILED, e.getMessage());
+
+                return new GridFinishedFuture<>(res);
+            }
+
+            req.clientId(clientIdAndSesTok.get1());
+            req.sessionToken(clientIdAndSesTok.get2());
+
+            if (log.isDebugEnabled())
+                log.debug("Next clientId and sessionToken were extracted from request: " +
+                    "[clientId="+req.clientId()+", sessionToken="+Arrays.toString(req.sessionToken())+"]");
+
             try {
                 subjCtx = authenticate(req);
 
@@ -228,7 +252,7 @@ public class GridRestProcessor extends GridProcessorAdapter {
 
                 if (ctx.security().enabled()) {
                     updateSession(req, subjCtx0);
-                    res.sessionTokenBytes(ZERO_BYTES);
+                    res.sessionTokenBytes(req.sessionToken());
                 }
 
                 interceptResponse(res, req);
@@ -238,6 +262,65 @@ public class GridRestProcessor extends GridProcessorAdapter {
         });
     }
 
+    // TODO add experation
+    private IgniteBiTuple<UUID, byte[]> clientIdAndSessionToken(GridRestRequest req) throws IgniteCheckedException {
+        UUID clientId = req.clientId();
+        byte[] sesTok = req.sessionToken();
+
+        if (F.isEmpty(sesTok) && clientId == null) {
+            clientId = UUID.randomUUID();
+
+            UUID sesTokUuid = UUID.randomUUID();
+
+            UUID oldSesTokUuid = clientId2SesToken.putIfAbsent(clientId, sesTokUuid);
+
+            assert oldSesTokUuid == null : "Error [oldSesTokUuid=" + oldSesTokUuid + "]";
+
+            sesToken2ClientId.put(sesTokUuid, clientId);
+
+            return new IgniteBiTuple<>(clientId, U.uuidToBytes(sesTokUuid));
+        }
+
+        if (F.isEmpty(sesTok) && clientId != null) {
+            UUID sesTokUuid = clientId2SesToken.get(clientId);
+
+            if (sesTokUuid == null) { /** First request with this clientId */
+                UUID newSesTokUuid = UUID.randomUUID();
+
+                UUID oldSesTokUuid = clientId2SesToken.putIfAbsent(clientId, newSesTokUuid);
+
+                sesTokUuid = oldSesTokUuid != null ? oldSesTokUuid : newSesTokUuid;
+
+                if (oldSesTokUuid == null)
+                    sesToken2ClientId.put(sesTokUuid, clientId);
+            }
+
+            return new IgniteBiTuple<>(clientId, U.uuidToBytes(sesTokUuid));
+        }
+
+        if (!F.isEmpty(sesTok) && clientId == null) {
+            UUID sesTokId = U.bytesToUuid(sesTok, 0);
+
+            clientId = sesToken2ClientId.get(sesTokId);
+
+            if (clientId == null)
+                throw new IgniteCheckedException("Failed to handle request. Unknown session token " +
+                    "(maybe expired session). [sessionToken=" + Arrays.toString(sesTok) + "]");
+
+            return new IgniteBiTuple<>(clientId, sesTok);
+        }
+
+        if (!F.isEmpty(sesTok) && clientId != null) {
+            UUID sesTokUuid = U.bytesToUuid(sesTok, 0);
+
+            if (sesToken2ClientId.get(sesTokUuid).equals(clientId) &&
+                clientId2SesToken.get(clientId).equals(sesTokUuid))
+                throw new IgniteCheckedException("Failed to handle request ");// TODO msg.
+        }
+
+        return new IgniteBiTuple<>(clientId, sesTok);
+    }
+
     /**
      * @param ctx Context.
      */
@@ -458,7 +541,10 @@ public class GridRestProcessor extends GridProcessorAdapter {
      */
     private SecurityContext authenticate(GridRestRequest req) throws IgniteCheckedException {
         UUID clientId = req.clientId();
-        SecurityContext secCtx = clientId == null ? null : sesMap.get(clientId);
+
+        assert clientId != null;
+
+        SecurityContext secCtx = sesMap.get(clientId);
 
         if (secCtx != null)
             return secCtx;
@@ -467,7 +553,7 @@ public class GridRestProcessor extends GridProcessorAdapter {
         AuthenticationContext authCtx = new AuthenticationContext();
 
         authCtx.subjectType(REMOTE_CLIENT);
-        authCtx.subjectId(req.clientId());
+        authCtx.subjectId(clientId);
 
         SecurityCredentials cred;
 
@@ -494,6 +580,7 @@ public class GridRestProcessor extends GridProcessorAdapter {
 
         SecurityContext subjCtx = ctx.security().authenticate(authCtx);
 
+        // TODO review
         if (subjCtx == null) {
             if (req.credentials() == null)
                 throw new IgniteCheckedException("Failed to authenticate remote client (secure session SPI not set?): " + req);
@@ -510,10 +597,8 @@ public class GridRestProcessor extends GridProcessorAdapter {
      * @param sCtx Security context.
      */
     private void updateSession(GridRestRequest req, SecurityContext sCtx) {
-        if (sCtx != null) {
-            UUID id = req.clientId();
-            sesMap.put(id, sCtx);
-        }
+        if (sCtx != null)
+            sesMap.put(req.clientId(), sCtx);
     }
 
     /**
@@ -665,4 +750,10 @@ public class GridRestProcessor extends GridProcessorAdapter {
         X.println(">>>   protosSize: " + protos.size());
         X.println(">>>   handlersSize: " + handlers.size());
     }
+
+    private static class Session {
+        UUID clientId;
+        UUID sesTokId;
+        SecurityContext secCtx;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/933940af/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/GridJettyRestHandler.java
----------------------------------------------------------------------
diff --git a/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/GridJettyRestHandler.java b/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/GridJettyRestHandler.java
index fac9818..8af0832 100644
--- a/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/GridJettyRestHandler.java
+++ b/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/GridJettyRestHandler.java
@@ -249,18 +249,25 @@ public class GridJettyRestHandler extends AbstractHandler {
         Map<String, Object> params = parameters(req);
 
         try {
+            U.dumpStack("JettyHandler#processRequest cmd=" + cmd + ", params=" + params + " req=" + req);
+
             GridRestRequest cmdReq = createRequest(cmd, params, req);
 
             if (log.isDebugEnabled())
                 log.debug("Initialized command request: " + cmdReq);
 
+            U.dumpStack(">>>>> Before handle");
+
             cmdRes = hnd.handle(cmdReq);
 
             if (cmdRes == null)
                 throw new IllegalStateException("Received null result from handler: " + hnd);
 
+            // TODO review.
             byte[] sesTok = cmdRes.sessionTokenBytes();
 
+            U.dumpStack(">>>>> TODO review. sesTok=" + Arrays.toString(sesTok));
+
             if (sesTok != null)
                 cmdRes.setSessionToken(U.byteArray2HexString(sesTok));
 
@@ -467,6 +474,8 @@ public class GridJettyRestHandler extends AbstractHandler {
         try {
             if (clientId != null)
                 restReq.clientId(UUID.fromString(clientId));
+//            else
+//                restReq.clientId(UUID.randomUUID());
         }
         catch (Exception ignored) {
             // Ignore invalid client id. Rest handler will process this logic.


[2/3] incubator-ignite git commit: # gg-10561: session

Posted by sb...@apache.org.
# gg-10561: session


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

Branch: refs/heads/ignite-gg-10561
Commit: f30b111ce85c172ee71a497001b38985b0156807
Parents: 933940a
Author: ashutak <as...@gridgain.com>
Authored: Wed Jul 22 16:46:25 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Wed Jul 22 16:46:25 2015 +0300

----------------------------------------------------------------------
 .../processors/rest/GridRestProcessor.java      | 146 +++++++++++--------
 1 file changed, 83 insertions(+), 63 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f30b111c/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java
index a76a119..bf6c0fb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java
@@ -76,7 +76,10 @@ public class GridRestProcessor extends GridProcessorAdapter {
     private final LongAdder8 workersCnt = new LongAdder8();
 
     /** SecurityContext map. */
-    private ConcurrentMap<UUID, SecurityContext> sesMap = new ConcurrentHashMap<>();
+    private ConcurrentMap<UUID, Session> clientId2Session = new ConcurrentHashMap<>();
+
+    /** SecurityContext map. */
+    private ConcurrentMap<UUID, Session> sesTokId2Session = new ConcurrentHashMap<>();
 
     /** Protocol handler. */
     private final GridRestProtocolHandler protoHnd = new GridRestProtocolHandler() {
@@ -89,12 +92,6 @@ public class GridRestProcessor extends GridProcessorAdapter {
         }
     };
 
-    /** Session token to client ID map. */
-    private final ConcurrentMap<UUID, UUID> sesToken2ClientId = new ConcurrentHashMap<>();
-
-    /** Client ID to session token map */
-    private final ConcurrentMap<UUID, UUID> clientId2SesToken = new ConcurrentHashMap<>();
-
     /**
      * @param req Request.
      * @return Future.
@@ -178,13 +175,11 @@ public class GridRestProcessor extends GridProcessorAdapter {
         if (log.isDebugEnabled())
             log.debug("Received request from client: " + req);
 
-        SecurityContext subjCtx = null;
+        Session ses = null;
 
         if (ctx.security().enabled()) {
-            IgniteBiTuple<UUID, byte[]> clientIdAndSesTok;
-
             try {
-                clientIdAndSesTok = clientIdAndSessionToken(req);
+                ses = session(req);
             }
             catch (IgniteCheckedException e) {
                 GridRestResponse res = new GridRestResponse(STATUS_FAILED, e.getMessage());
@@ -192,24 +187,26 @@ public class GridRestProcessor extends GridProcessorAdapter {
                 return new GridFinishedFuture<>(res);
             }
 
-            req.clientId(clientIdAndSesTok.get1());
-            req.sessionToken(clientIdAndSesTok.get2());
+            assert ses != null;
+
+            req.clientId(ses.clientId);
+            req.sessionToken(ses.sesTok());
 
             if (log.isDebugEnabled())
                 log.debug("Next clientId and sessionToken were extracted from request: " +
                     "[clientId="+req.clientId()+", sessionToken="+Arrays.toString(req.sessionToken())+"]");
 
             try {
-                subjCtx = authenticate(req);
+                if (ses.secCtx == null)
+                    ses.secCtx = authenticate(req);
 
-                authorize(req, subjCtx);
+                authorize(req, ses.secCtx);
             }
             catch (SecurityException e) {
-                assert subjCtx != null;
+                assert ses.secCtx != null;
 
                 GridRestResponse res = new GridRestResponse(STATUS_SECURITY_CHECK_FAILED, e.getMessage());
 
-                updateSession(req, subjCtx);
                 res.sessionTokenBytes(ZERO_BYTES);
 
                 return new GridFinishedFuture<>(res);
@@ -229,7 +226,7 @@ public class GridRestProcessor extends GridProcessorAdapter {
             return new GridFinishedFuture<>(
                 new IgniteCheckedException("Failed to find registered handler for command: " + req.command()));
 
-        final SecurityContext subjCtx0 = subjCtx;
+        final SecurityContext subjCtx0 = ses == null ? null : ses.secCtx;
 
         return res.chain(new C1<IgniteInternalFuture<GridRestResponse>, GridRestResponse>() {
             @Override public GridRestResponse apply(IgniteInternalFuture<GridRestResponse> f) {
@@ -251,7 +248,8 @@ public class GridRestProcessor extends GridProcessorAdapter {
                 assert res != null;
 
                 if (ctx.security().enabled()) {
-                    updateSession(req, subjCtx0);
+                    // TODO review. Why we update it here, not earlier?
+//                    updateSession(req, subjCtx0);
                     res.sessionTokenBytes(req.sessionToken());
                 }
 
@@ -262,63 +260,64 @@ public class GridRestProcessor extends GridProcessorAdapter {
         });
     }
 
-    // TODO add experation
-    private IgniteBiTuple<UUID, byte[]> clientIdAndSessionToken(GridRestRequest req) throws IgniteCheckedException {
-        UUID clientId = req.clientId();
-        byte[] sesTok = req.sessionToken();
+    /**
+     * // TODO expiration
+     * @param req Request.
+     * @return Not null session.
+     * @throws IgniteCheckedException If failed.
+     */
+    private Session session(GridRestRequest req) throws IgniteCheckedException {
+        final UUID clientId = req.clientId();
+        final byte[] sesTok = req.sessionToken();
 
         if (F.isEmpty(sesTok) && clientId == null) {
-            clientId = UUID.randomUUID();
-
-            UUID sesTokUuid = UUID.randomUUID();
+            Session ses = new Session();
+            ses.clientId = UUID.randomUUID();
+            ses.sesTokId = UUID.randomUUID();
 
-            UUID oldSesTokUuid = clientId2SesToken.putIfAbsent(clientId, sesTokUuid);
+            clientId2Session.put(ses.clientId, ses);
+            sesTokId2Session.put(ses.sesTokId, ses);
 
-            assert oldSesTokUuid == null : "Error [oldSesTokUuid=" + oldSesTokUuid + "]";
+            // TODO add checks.
 
-            sesToken2ClientId.put(sesTokUuid, clientId);
-
-            return new IgniteBiTuple<>(clientId, U.uuidToBytes(sesTokUuid));
+            return ses;
         }
 
         if (F.isEmpty(sesTok) && clientId != null) {
-            UUID sesTokUuid = clientId2SesToken.get(clientId);
+            Session ses = clientId2Session.get(clientId);
 
-            if (sesTokUuid == null) { /** First request with this clientId */
-                UUID newSesTokUuid = UUID.randomUUID();
+            if (ses == null) { /** First request with this clientId */
+                ses = new Session();
+                ses.clientId = clientId;
+                ses.sesTokId = UUID.randomUUID();
 
-                UUID oldSesTokUuid = clientId2SesToken.putIfAbsent(clientId, newSesTokUuid);
+                clientId2Session.put(ses.clientId, ses);
+                sesTokId2Session.put(ses.sesTokId, ses);
 
-                sesTokUuid = oldSesTokUuid != null ? oldSesTokUuid : newSesTokUuid;
+                // TODO add checks.
 
-                if (oldSesTokUuid == null)
-                    sesToken2ClientId.put(sesTokUuid, clientId);
+                return ses;
             }
 
-            return new IgniteBiTuple<>(clientId, U.uuidToBytes(sesTokUuid));
+            return ses;
         }
 
         if (!F.isEmpty(sesTok) && clientId == null) {
             UUID sesTokId = U.bytesToUuid(sesTok, 0);
 
-            clientId = sesToken2ClientId.get(sesTokId);
+            Session ses = sesTokId2Session.get(sesTokId);
 
-            if (clientId == null)
+            if (ses == null)
                 throw new IgniteCheckedException("Failed to handle request. Unknown session token " +
-                    "(maybe expired session). [sessionToken=" + Arrays.toString(sesTok) + "]");
+                    "(maybe expired session). [sessionToken=" + U.byteArray2HexString(sesTok) + "]");
 
-            return new IgniteBiTuple<>(clientId, sesTok);
+            return ses;
         }
 
-        if (!F.isEmpty(sesTok) && clientId != null) {
-            UUID sesTokUuid = U.bytesToUuid(sesTok, 0);
-
-            if (sesToken2ClientId.get(sesTokUuid).equals(clientId) &&
-                clientId2SesToken.get(clientId).equals(sesTokUuid))
-                throw new IgniteCheckedException("Failed to handle request ");// TODO msg.
-        }
+        if (!F.isEmpty(sesTok) && clientId != null)
+            throw new IgniteCheckedException("Failed to handle request. Unsupported case.");
 
-        return new IgniteBiTuple<>(clientId, sesTok);
+        throw new IgniteCheckedException("Failed to handle request (Unreachable state).");
     }
 
     /**
@@ -540,20 +539,11 @@ public class GridRestProcessor extends GridProcessorAdapter {
      * @throws IgniteCheckedException If authentication failed.
      */
     private SecurityContext authenticate(GridRestRequest req) throws IgniteCheckedException {
-        UUID clientId = req.clientId();
-
-        assert clientId != null;
-
-        SecurityContext secCtx = sesMap.get(clientId);
-
-        if (secCtx != null)
-            return secCtx;
-
         // Authenticate client if invalid session.
         AuthenticationContext authCtx = new AuthenticationContext();
 
         authCtx.subjectType(REMOTE_CLIENT);
-        authCtx.subjectId(clientId);
+        authCtx.subjectId(req.clientId());
 
         SecurityCredentials cred;
 
@@ -597,8 +587,10 @@ public class GridRestProcessor extends GridProcessorAdapter {
      * @param sCtx Security context.
      */
     private void updateSession(GridRestRequest req, SecurityContext sCtx) {
-        if (sCtx != null)
-            sesMap.put(req.clientId(), sCtx);
+//        if (sCtx != null) {
+//            sesMap.put(req.clientId(), sCtx);
+//
+//        }
     }
 
     /**
@@ -755,5 +747,33 @@ public class GridRestProcessor extends GridProcessorAdapter {
         UUID clientId;
         UUID sesTokId;
         SecurityContext secCtx;
+
+        byte[] sesTok(){
+            return U.uuidToBytes(sesTokId);
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+            if (!(o instanceof Session))
+                return false;
+
+            Session ses = (Session)o;
+
+            if (clientId != null ? !clientId.equals(ses.clientId) : ses.clientId != null)
+                return false;
+            if (sesTokId != null ? !sesTokId.equals(ses.sesTokId) : ses.sesTokId != null)
+                return false;
+
+            return true;
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            int res = clientId != null ? clientId.hashCode() : 0;
+            res = 31 * res + (sesTokId != null ? sesTokId.hashCode() : 0);
+            return res;
+        }
     }
 }


[3/3] incubator-ignite git commit: # gg-10561: tests and fix session

Posted by sb...@apache.org.
# gg-10561: tests and fix session


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

Branch: refs/heads/ignite-gg-10561
Commit: 482cfbb8c733305e4d80f1495dd79865682ee4c6
Parents: f30b111
Author: ashutak <as...@gridgain.com>
Authored: Wed Jul 22 19:12:15 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Wed Jul 22 19:12:15 2015 +0300

----------------------------------------------------------------------
 .../processors/rest/GridRestProcessor.java      | 24 ++++++++++++--------
 1 file changed, 15 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/482cfbb8/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java
index bf6c0fb..60c7ef4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java
@@ -278,8 +278,6 @@ public class GridRestProcessor extends GridProcessorAdapter {
             clientId2Session.put(ses.clientId, ses);
             sesTokId2Session.put(ses.sesTokId, ses);
 
-            // TODO add checks.
-
             return ses;
         }
 
@@ -291,12 +289,12 @@ public class GridRestProcessor extends GridProcessorAdapter {
                 ses.clientId = clientId;
                 ses.sesTokId = UUID.randomUUID();
 
-                clientId2Session.put(ses.clientId, ses);
-                sesTokId2Session.put(ses.sesTokId, ses);
+                Session oldSes = clientId2Session.putIfAbsent(ses.clientId, ses);
 
-                // TODO add checks.
+                if (oldSes == null)
+                    sesTokId2Session.put(ses.sesTokId, ses);
 
-                return ses;
+                return oldSes == null ? ses : oldSes;
             }
 
             return ses;
@@ -308,14 +306,22 @@ public class GridRestProcessor extends GridProcessorAdapter {
             Session ses = sesTokId2Session.get(sesTokId);
 
             if (ses == null)
-                throw new IgniteCheckedException("Failed to handle request. Unknown session token " +
+                throw new IgniteCheckedException("Faile to handle request. Unknown session token " +
                     "(maybe expired session). [sessionToken=" + U.byteArray2HexString(sesTok) + "]");
 
             return ses;
         }
 
-        if (!F.isEmpty(sesTok) && clientId != null)
-            throw new IgniteCheckedException("Failed to handle request. Unsupported case.");
+        if (!F.isEmpty(sesTok) && clientId != null) {
+            Session ses1 = sesTokId2Session.get(U.bytesToUuid(sesTok, 0));
+            Session ses2 = clientId2Session.get(clientId);
+
+            if (ses1 == null || ses2 == null || !ses1.equals(ses2))
+                throw new IgniteCheckedException("Failed to handle request. " +
+                    "Unsupported case (use one: clientId or session token)");
+
+            return ses1;
+        }
 
         throw new IgniteCheckedException("Failed to handle request (Unreachable state).");
     }