You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by an...@apache.org on 2016/11/16 10:58:16 UTC

[2/3] ignite git commit: IGNITE-3066 Limited Redis protocol implementation. - Fixes #1212.

IGNITE-3066 Limited Redis protocol implementation. - Fixes #1212.

Signed-off-by: Andrey Novikov <an...@apache.org>


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

Branch: refs/heads/master
Commit: b8a2b8ed81175887dd11d8f6b719c9f561d6a028
Parents: 2c64dd0
Author: shtykh_roman <rs...@yahoo.com>
Authored: Wed Nov 16 15:41:15 2016 +0700
Committer: Andrey Novikov <an...@apache.org>
Committed: Wed Nov 16 15:41:15 2016 +0700

----------------------------------------------------------------------
 modules/clients/pom.xml                         |  17 +-
 .../tcp/redis/RedisProtocolSelfTest.java        | 465 +++++++++++++++++++
 .../handlers/redis/GridRedisCommandHandler.java |  39 ++
 .../GridRedisConnectionCommandHandler.java      |  71 +++
 .../redis/GridRedisRestCommandHandler.java      | 102 ++++
 .../exception/GridRedisGenericException.java    |  37 ++
 .../redis/exception/GridRedisTypeException.java |  37 ++
 .../redis/key/GridRedisDelCommandHandler.java   |  89 ++++
 .../key/GridRedisExistsCommandHandler.java      |  88 ++++
 .../server/GridRedisDbSizeCommandHandler.java   |  75 +++
 .../string/GridRedisAppendCommandHandler.java   | 112 +++++
 .../string/GridRedisGetCommandHandler.java      |  81 ++++
 .../string/GridRedisGetRangeCommandHandler.java | 122 +++++
 .../string/GridRedisGetSetCommandHandler.java   |  89 ++++
 .../string/GridRedisIncrDecrCommandHandler.java | 134 ++++++
 .../string/GridRedisMGetCommandHandler.java     |  88 ++++
 .../string/GridRedisMSetCommandHandler.java     |  86 ++++
 .../string/GridRedisSetCommandHandler.java      | 130 ++++++
 .../string/GridRedisSetRangeCommandHandler.java | 141 ++++++
 .../string/GridRedisStrlenCommandHandler.java   |  83 ++++
 .../protocols/tcp/GridClientPacketType.java     |   3 +
 .../protocols/tcp/GridTcpRestNioListener.java   |  14 +-
 .../rest/protocols/tcp/GridTcpRestParser.java   |  39 +-
 .../protocols/tcp/redis/GridRedisCommand.java   |  80 ++++
 .../protocols/tcp/redis/GridRedisMessage.java   | 219 +++++++++
 .../tcp/redis/GridRedisNioListener.java         | 146 ++++++
 .../tcp/redis/GridRedisProtocolParser.java      | 314 +++++++++++++
 27 files changed, 2883 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b8a2b8ed/modules/clients/pom.xml
----------------------------------------------------------------------
diff --git a/modules/clients/pom.xml b/modules/clients/pom.xml
index e445229..2e29cc0 100644
--- a/modules/clients/pom.xml
+++ b/modules/clients/pom.xml
@@ -20,7 +20,8 @@
 <!--
     POM file.
 -->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
     <modelVersion>4.0.0</modelVersion>
 
     <parent>
@@ -55,6 +56,13 @@
         </dependency>
 
         <dependency>
+            <groupId>redis.clients</groupId>
+            <artifactId>jedis</artifactId>
+            <version>2.9.0</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
             <groupId>org.apache.ignite</groupId>
             <artifactId>ignite-spring</artifactId>
             <version>${project.version}</version>
@@ -160,10 +168,11 @@
                                 <configuration>
                                     <target>
                                         <exec executable="${doxygen.exec}" searchpath="true" dir="../platforms/dotnet/">
-                                            <arg value="Apache.Ignite.dxg" />
+                                            <arg value="Apache.Ignite.dxg"/>
                                         </exec>
 
-                                        <copy file="../../assembly/docfiles/ignite_logo.png" todir="target/dotnetdoc/html" />
+                                        <copy file="../../assembly/docfiles/ignite_logo.png"
+                                              todir="target/dotnetdoc/html"/>
                                     </target>
                                 </configuration>
                             </execution>
@@ -177,7 +186,7 @@
                                 <configuration>
                                     <target>
                                         <exec executable="${doxygen.exec}" searchpath="true" dir="../platforms/cpp/">
-                                            <arg value="cpp.dxg" />
+                                            <arg value="cpp.dxg"/>
                                         </exec>
                                     </target>
                                 </configuration>

http://git-wip-us.apache.org/repos/asf/ignite/blob/b8a2b8ed/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/protocols/tcp/redis/RedisProtocolSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/protocols/tcp/redis/RedisProtocolSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/protocols/tcp/redis/RedisProtocolSelfTest.java
new file mode 100644
index 0000000..bd20b27
--- /dev/null
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/protocols/tcp/redis/RedisProtocolSelfTest.java
@@ -0,0 +1,465 @@
+/*
+ * 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.rest.protocols.tcp.redis;
+
+import java.math.BigInteger;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.ConnectorConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Assert;
+import redis.clients.jedis.Jedis;
+import redis.clients.jedis.JedisPool;
+import redis.clients.jedis.JedisPoolConfig;
+import redis.clients.jedis.exceptions.JedisDataException;
+
+/**
+ * Tests for Redis protocol.
+ */
+public class RedisProtocolSelfTest extends GridCommonAbstractTest {
+    /** Grid count. */
+    private static final int GRID_CNT = 2;
+
+    /** IP finder. */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** Local host. */
+    private static final String HOST = "127.0.0.1";
+
+    /** Port. */
+    private static final int PORT = 6379;
+
+    /** Pool. */
+    private static JedisPool pool;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        startGrids(gridCount());
+
+        JedisPoolConfig jedisPoolCfg = new JedisPoolConfig();
+
+        jedisPoolCfg.setMaxWaitMillis(10000);
+        jedisPoolCfg.setMaxIdle(100);
+        jedisPoolCfg.setMinIdle(1);
+        jedisPoolCfg.setNumTestsPerEvictionRun(10);
+        jedisPoolCfg.setTestOnBorrow(true);
+        jedisPoolCfg.setTestOnReturn(true);
+        jedisPoolCfg.setTestWhileIdle(true);
+        jedisPoolCfg.setTimeBetweenEvictionRunsMillis(30000);
+
+        pool = new JedisPool(jedisPoolCfg, HOST, PORT, 10000);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+
+        pool.destroy();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setLocalHost(HOST);
+
+        assert cfg.getConnectorConfiguration() == null;
+
+        ConnectorConfiguration redisCfg = new ConnectorConfiguration();
+
+        redisCfg.setHost(HOST);
+        redisCfg.setPort(PORT);
+
+        cfg.setConnectorConfiguration(redisCfg);
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(IP_FINDER);
+
+        cfg.setDiscoverySpi(disco);
+
+        CacheConfiguration ccfg = defaultCacheConfiguration();
+
+        ccfg.setStatisticsEnabled(true);
+        ccfg.setIndexedTypes(String.class, String.class);
+
+        cfg.setCacheConfiguration(ccfg);
+
+        return cfg;
+    }
+
+    /**
+     * @return Cache.
+     */
+    @Override protected <K, V> IgniteCache<K, V> jcache() {
+        return grid(0).cache(null);
+    }
+
+    /** {@inheritDoc} */
+    protected int gridCount() {
+        return GRID_CNT;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        assert grid(0).cluster().nodes().size() == gridCount();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        jcache().clear();
+
+        assertTrue(jcache().localSize() == 0);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPing() throws Exception {
+        try (Jedis jedis = pool.getResource()) {
+            Assert.assertEquals("PONG", jedis.ping());
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testEcho() throws Exception {
+        try (Jedis jedis = pool.getResource()) {
+            Assert.assertEquals("Hello, grid!", jedis.echo("Hello, grid!"));
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testGet() throws Exception {
+        try (Jedis jedis = pool.getResource()) {
+            jcache().put("getKey1", "getVal1");
+
+            Assert.assertEquals("getVal1", jedis.get("getKey1"));
+            Assert.assertNull(jedis.get("wrongKey"));
+
+            jcache().put("setDataTypeKey", new HashSet<String>(Arrays.asList("1", "2")));
+            try {
+                jedis.get("setDataTypeKey");
+
+                assert false : "Exception has to be thrown!";
+            }
+            catch (JedisDataException e) {
+                assertTrue(e.getMessage().startsWith("WRONGTYPE"));
+            }
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testGetSet() throws Exception {
+        try (Jedis jedis = pool.getResource()) {
+            jcache().put("getSetKey1", "1");
+
+            Assert.assertEquals("1", jedis.getSet("getSetKey1", "0"));
+            Assert.assertNull(jedis.get("getSetNonExistingKey"));
+
+            jcache().put("setDataTypeKey", new HashSet<String>(Arrays.asList("1", "2")));
+            try {
+                jedis.getSet("setDataTypeKey", "0");
+
+                assert false : "Exception has to be thrown!";
+            }
+            catch (JedisDataException e) {
+                assertTrue(e.getMessage().startsWith("WRONGTYPE"));
+            }
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMGet() throws Exception {
+        try (Jedis jedis = pool.getResource()) {
+            jcache().put("getKey1", "getVal1");
+            jcache().put("getKey2", 0);
+
+            List<String> result = jedis.mget("getKey1", "getKey2", "wrongKey");
+            Assert.assertTrue(result.contains("getVal1"));
+            Assert.assertTrue(result.contains("0"));
+
+            // not supported.
+//            fail("Incompatible! getAll() does not return null values!");
+//            Assert.assertTrue(result.contains("nil"));
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSet() throws Exception {
+        try (Jedis jedis = pool.getResource()) {
+            jedis.set("setKey1", "1");
+            jedis.set("setKey2".getBytes(), "b0".getBytes());
+
+            Assert.assertEquals("1", jcache().get("setKey1"));
+            Assert.assertEquals("b0", jcache().get("setKey2"));
+
+            // test options.
+            jedis.set("setKey1", "2", "nx");
+            jedis.set("setKey3", "3", "nx");
+            Assert.assertEquals("1", jcache().get("setKey1"));
+            Assert.assertEquals("3", jcache().get("setKey3"));
+
+            jedis.set("setKey1", "2", "xx");
+            jedis.set("setKey4", "4", "xx");
+            Assert.assertEquals("2", jcache().get("setKey1"));
+            Assert.assertNull(jcache().get("setKey4"));
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMSet() throws Exception {
+        try (Jedis jedis = pool.getResource()) {
+            jedis.mset("setKey1", "1", "setKey2", "2");
+
+            Assert.assertEquals("1", jcache().get("setKey1"));
+            Assert.assertEquals("2", jcache().get("setKey2"));
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testIncrDecr() throws Exception {
+        try (Jedis jedis = pool.getResource()) {
+            Assert.assertEquals(1, (long)jedis.incr("newKeyIncr"));
+            Assert.assertEquals(-1, (long)jedis.decr("newKeyDecr"));
+
+            jcache().put("incrKey1", 1L);
+            Assert.assertEquals(2L, (long)jedis.incr("incrKey1"));
+            jcache().put("decrKey1", 1L);
+            Assert.assertEquals(0L, (long)jedis.decr("decrKey1"));
+
+            jcache().put("nonInt", "abc");
+            try {
+                jedis.incr("nonInt");
+
+                assert false : "Exception has to be thrown!";
+            }
+            catch (JedisDataException e) {
+                assertTrue(e.getMessage().startsWith("ERR"));
+            }
+            try {
+                jedis.decr("nonInt");
+
+                assert false : "Exception has to be thrown!";
+            }
+            catch (JedisDataException e) {
+                assertTrue(e.getMessage().startsWith("ERR"));
+            }
+
+            jcache().put("outOfRange", new BigInteger("234293482390480948029348230948"));
+            try {
+                jedis.incr("outOfRange");
+
+                assert false : "Exception has to be thrown!";
+            }
+            catch (JedisDataException e) {
+                assertTrue(e.getMessage().startsWith("ERR"));
+            }
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testIncrDecrBy() throws Exception {
+        try (Jedis jedis = pool.getResource()) {
+            Assert.assertEquals(2, (long)jedis.incrBy("newKeyIncr1", 2));
+            Assert.assertEquals(-2, (long)jedis.decrBy("newKeyDecr1", 2));
+
+            jcache().put("incrKey2", 1L);
+            Assert.assertEquals(3L, (long)jedis.incrBy("incrKey2", 2));
+            jcache().put("decrKey2", 2L);
+            Assert.assertEquals(0L, (long)jedis.decrBy("decrKey2", 2));
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAppend() throws Exception {
+        try (Jedis jedis = pool.getResource()) {
+            Assert.assertEquals(5, (long)jedis.append("appendKey1", "Hello"));
+            Assert.assertEquals(12, (long)jedis.append("appendKey1", " World!"));
+
+            jcache().put("setDataTypeKey", new HashSet<String>(Arrays.asList("1", "2")));
+            try {
+                jedis.append("setDataTypeKey", "");
+
+                assert false : "Exception has to be thrown!";
+            }
+            catch (JedisDataException e) {
+                assertTrue(e.getMessage().startsWith("WRONGTYPE"));
+            }
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStrlen() throws Exception {
+        try (Jedis jedis = pool.getResource()) {
+            Assert.assertEquals(0, (long)jedis.strlen("strlenKeyNonExisting"));
+
+            jcache().put("strlenKey", "abc");
+            Assert.assertEquals(3, (long)jedis.strlen("strlenKey"));
+
+            jcache().put("setDataTypeKey", new HashSet<String>(Arrays.asList("1", "2")));
+            try {
+                jedis.strlen("setDataTypeKey");
+
+                assert false : "Exception has to be thrown!";
+            }
+            catch (JedisDataException e) {
+                assertTrue(e.getMessage().startsWith("WRONGTYPE"));
+            }
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSetRange() throws Exception {
+        try (Jedis jedis = pool.getResource()) {
+            Assert.assertEquals(0, (long)jedis.setrange("setRangeKey1", 0, ""));
+
+            jcache().put("setRangeKey2", "abc");
+            Assert.assertEquals(3, (long)jedis.setrange("setRangeKey2", 0, ""));
+
+            Assert.assertEquals(3, (long)jedis.setrange("setRangeKeyPadded", 2, "a"));
+
+            try {
+                jedis.setrange("setRangeKeyWrongOffset", -1, "a");
+
+                assert false : "Exception has to be thrown!";
+            }
+            catch (JedisDataException e) {
+                assertTrue(e.getMessage().startsWith("ERR"));
+            }
+
+            try {
+                jedis.setrange("setRangeKeyWrongOffset2", 536870911, "a");
+
+                assert false : "Exception has to be thrown!";
+            }
+            catch (JedisDataException e) {
+                assertTrue(e.getMessage().startsWith("ERR"));
+            }
+
+            jcache().put("setRangeKey3", "Hello World");
+            Assert.assertEquals(11, (long)jedis.setrange("setRangeKey3", 6, "Redis"));
+
+            jcache().put("setDataTypeKey", new HashSet<String>(Arrays.asList("1", "2")));
+            try {
+                jedis.setrange("setDataTypeKey", 0, "Redis");
+
+                assert false : "Exception has to be thrown!";
+            }
+            catch (JedisDataException e) {
+                assertTrue(e.getMessage().startsWith("WRONGTYPE"));
+            }
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testGetRange() throws Exception {
+        try (Jedis jedis = pool.getResource()) {
+            Assert.assertEquals("", jedis.getrange("getRangeKeyNonExisting", 0, 0));
+
+            jcache().put("getRangeKey", "This is a string");
+            Assert.assertEquals("This", jedis.getrange("getRangeKey", 0, 3));
+            Assert.assertEquals("ing", jedis.getrange("getRangeKey", -3, -1));
+            Assert.assertEquals("This is a string", jedis.getrange("getRangeKey", 0, -1));
+            Assert.assertEquals("string", jedis.getrange("getRangeKey", 10, 100));
+
+            jcache().put("setDataTypeKey", new HashSet<String>(Arrays.asList("1", "2")));
+            try {
+                jedis.getrange("setDataTypeKey", 0, 1);
+
+                assert false : "Exception has to be thrown!";
+            }
+            catch (JedisDataException e) {
+                assertTrue(e.getMessage().startsWith("WRONGTYPE"));
+            }
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDel() throws Exception {
+        jcache().put("delKey1", "abc");
+        jcache().put("delKey2", "abcd");
+        try (Jedis jedis = pool.getResource()) {
+            // Should return the number of actually deleted entries.
+//            Assert.assertEquals(0, (long)jedis.del("nonExistingDelKey"));
+            Assert.assertEquals(2, (long)jedis.del("delKey1", "delKey2"));
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testExists() throws Exception {
+        jcache().put("existsKey1", "abc");
+        jcache().put("existsKey2", "abcd");
+        try (Jedis jedis = pool.getResource()) {
+            Assert.assertFalse(jedis.exists("nonExistingDelKey"));
+            Assert.assertEquals(2, (long)jedis.exists("existsKey1", "existsKey2"));
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDbSize() throws Exception {
+        try (Jedis jedis = pool.getResource()) {
+            Assert.assertEquals(0, (long)jedis.dbSize());
+
+            jcache().putAll(new HashMap<Integer, Integer>() {
+                {
+                    for (int i = 0; i < 100; i++)
+                        put(i, i);
+                }
+            });
+            Assert.assertEquals(100, (long)jedis.dbSize());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b8a2b8ed/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/GridRedisCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/GridRedisCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/GridRedisCommandHandler.java
new file mode 100644
index 0000000..2e2a048
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/GridRedisCommandHandler.java
@@ -0,0 +1,39 @@
+/*
+ * 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.rest.handlers.redis;
+
+import java.util.Collection;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisCommand;
+import org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisMessage;
+
+/**
+ * Command handler.
+ */
+public interface GridRedisCommandHandler {
+    /**
+     * @return Collection of supported commands.
+     */
+    public Collection<GridRedisCommand> supportedCommands();
+
+    /**
+     * @param msg Request message.
+     * @return Future.
+     */
+    public IgniteInternalFuture<GridRedisMessage> handleAsync(GridRedisMessage msg);
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b8a2b8ed/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/GridRedisConnectionCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/GridRedisConnectionCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/GridRedisConnectionCommandHandler.java
new file mode 100644
index 0000000..b8a042d
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/GridRedisConnectionCommandHandler.java
@@ -0,0 +1,71 @@
+/*
+ * 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.rest.handlers.redis;
+
+import java.util.Collection;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisCommand;
+import org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisMessage;
+import org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisProtocolParser;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import static org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisCommand.ECHO;
+import static org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisCommand.PING;
+import static org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisCommand.QUIT;
+
+/**
+ * Redis connection handler.
+ */
+public class GridRedisConnectionCommandHandler implements GridRedisCommandHandler {
+    /** Supported commands. */
+    private static final Collection<GridRedisCommand> SUPPORTED_COMMANDS = U.sealList(
+        PING,
+        QUIT,
+        ECHO
+    );
+
+    /** PONG response to PING. */
+    private static final String PONG = "PONG";
+
+    /** {@inheritDoc} */
+    @Override public Collection<GridRedisCommand> supportedCommands() {
+        return SUPPORTED_COMMANDS;
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteInternalFuture<GridRedisMessage> handleAsync(GridRedisMessage msg) {
+        assert msg != null;
+
+        switch (msg.command()) {
+            case PING:
+                msg.setResponse(GridRedisProtocolParser.toSimpleString(PONG));
+                return new GridFinishedFuture<>(msg);
+
+            case QUIT:
+                msg.setResponse(GridRedisProtocolParser.oKString());
+                return new GridFinishedFuture<>(msg);
+
+            case ECHO:
+                msg.setResponse(GridRedisProtocolParser.toSimpleString(msg.key()));
+                return new GridFinishedFuture<>(msg);
+        }
+
+        return new GridFinishedFuture<>();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b8a2b8ed/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/GridRedisRestCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/GridRedisRestCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/GridRedisRestCommandHandler.java
new file mode 100644
index 0000000..696fea5
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/GridRedisRestCommandHandler.java
@@ -0,0 +1,102 @@
+/*
+ * 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.rest.handlers.redis;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.processors.rest.GridRestProtocolHandler;
+import org.apache.ignite.internal.processors.rest.GridRestResponse;
+import org.apache.ignite.internal.processors.rest.handlers.redis.exception.GridRedisTypeException;
+import org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisMessage;
+import org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisProtocolParser;
+import org.apache.ignite.internal.processors.rest.request.GridRestRequest;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.typedef.CX1;
+
+/**
+ * Redis command handler done via REST.
+ */
+public abstract class GridRedisRestCommandHandler implements GridRedisCommandHandler {
+    /** Logger. */
+    protected final IgniteLogger log;
+
+    /** REST protocol handler. */
+    protected final GridRestProtocolHandler hnd;
+
+    /**
+     * Constructor.
+     *
+     * @param hnd REST protocol handler.
+     */
+    public GridRedisRestCommandHandler(final IgniteLogger log, final GridRestProtocolHandler hnd) {
+        this.log = log;
+        this.hnd = hnd;
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteInternalFuture<GridRedisMessage> handleAsync(final GridRedisMessage msg) {
+        assert msg != null;
+
+        try {
+            return hnd.handleAsync(asRestRequest(msg))
+                .chain(new CX1<IgniteInternalFuture<GridRestResponse>, GridRedisMessage>() {
+                    @Override
+                    public GridRedisMessage applyx(IgniteInternalFuture<GridRestResponse> f)
+                        throws IgniteCheckedException {
+                        GridRestResponse restRes = f.get();
+
+                        if (restRes.getSuccessStatus() == GridRestResponse.STATUS_SUCCESS)
+                            msg.setResponse(makeResponse(restRes, msg.auxMKeys()));
+                        else
+                            msg.setResponse(GridRedisProtocolParser.toGenericError("Operation error"));
+
+                        return msg;
+                    }
+                });
+        }
+        catch (IgniteCheckedException e) {
+            if (e instanceof GridRedisTypeException)
+                msg.setResponse(GridRedisProtocolParser.toTypeError(e.getMessage()));
+            else
+                msg.setResponse(GridRedisProtocolParser.toGenericError(e.getMessage()));
+
+            return new GridFinishedFuture<>(msg);
+        }
+    }
+
+    /**
+     * Converts {@link GridRedisMessage} to {@link GridRestRequest}.
+     *
+     * @param msg {@link GridRedisMessage}
+     * @return {@link GridRestRequest}
+     * @throws IgniteCheckedException If fails.
+     */
+    public abstract GridRestRequest asRestRequest(GridRedisMessage msg) throws IgniteCheckedException;
+
+    /**
+     * Prepares a response according to the request.
+     *
+     * @param resp REST response.
+     * @param params Auxiliary parameters.
+     * @return Response for the command.
+     */
+    public abstract ByteBuffer makeResponse(GridRestResponse resp, List<String> params);
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b8a2b8ed/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/exception/GridRedisGenericException.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/exception/GridRedisGenericException.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/exception/GridRedisGenericException.java
new file mode 100644
index 0000000..d45dbbc
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/exception/GridRedisGenericException.java
@@ -0,0 +1,37 @@
+/*
+ * 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.rest.handlers.redis.exception;
+
+import org.apache.ignite.IgniteCheckedException;
+
+/**
+ * Generic Redis protocol exception.
+ */
+public class GridRedisGenericException extends IgniteCheckedException {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /**
+     * Creates a generic exception with given error message.
+     *
+     * @param msg Error message.
+     */
+    public GridRedisGenericException(String msg) {
+        super(msg);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b8a2b8ed/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/exception/GridRedisTypeException.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/exception/GridRedisTypeException.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/exception/GridRedisTypeException.java
new file mode 100644
index 0000000..69f425c
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/exception/GridRedisTypeException.java
@@ -0,0 +1,37 @@
+/*
+ * 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.rest.handlers.redis.exception;
+
+import org.apache.ignite.IgniteCheckedException;
+
+/**
+ * Exception on operation on the wrong data type.
+ */
+public class GridRedisTypeException extends IgniteCheckedException {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /**
+     * Creates a type exception with given error message.
+     *
+     * @param msg Error message.
+     */
+    public GridRedisTypeException(String msg) {
+        super(msg);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b8a2b8ed/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/key/GridRedisDelCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/key/GridRedisDelCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/key/GridRedisDelCommandHandler.java
new file mode 100644
index 0000000..9447fe1
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/key/GridRedisDelCommandHandler.java
@@ -0,0 +1,89 @@
+/*
+ * 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.rest.handlers.redis.key;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.rest.GridRestProtocolHandler;
+import org.apache.ignite.internal.processors.rest.GridRestResponse;
+import org.apache.ignite.internal.processors.rest.handlers.redis.GridRedisRestCommandHandler;
+import org.apache.ignite.internal.processors.rest.handlers.redis.exception.GridRedisGenericException;
+import org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisCommand;
+import org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisMessage;
+import org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisProtocolParser;
+import org.apache.ignite.internal.processors.rest.request.GridRestCacheRequest;
+import org.apache.ignite.internal.processors.rest.request.GridRestRequest;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import static org.apache.ignite.internal.processors.rest.GridRestCommand.CACHE_REMOVE_ALL;
+import static org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisCommand.DEL;
+
+/**
+ * Redis DEL command handler.
+ */
+public class GridRedisDelCommandHandler extends GridRedisRestCommandHandler {
+    /** Supported commands. */
+    private static final Collection<GridRedisCommand> SUPPORTED_COMMANDS = U.sealList(
+        DEL
+    );
+
+    /** {@inheritDoc} */
+    public GridRedisDelCommandHandler(final IgniteLogger log, final GridRestProtocolHandler hnd) {
+        super(log, hnd);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<GridRedisCommand> supportedCommands() {
+        return SUPPORTED_COMMANDS;
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridRestRequest asRestRequest(GridRedisMessage msg) throws IgniteCheckedException {
+        assert msg != null;
+
+        if (msg.messageSize() < 2)
+            throw new GridRedisGenericException("Wrong number of arguments");
+
+        GridRestCacheRequest restReq = new GridRestCacheRequest();
+
+        restReq.clientId(msg.clientId());
+        restReq.key(msg.key());
+        restReq.command(CACHE_REMOVE_ALL);
+
+        List<String> keys = msg.auxMKeys();
+        Map<Object, Object> mget = U.newHashMap(keys.size());
+
+        for (String key : keys)
+            mget.put(key, null);
+
+        restReq.values(mget);
+
+        return restReq;
+    }
+
+    /** {@inheritDoc} */
+    @Override public ByteBuffer makeResponse(final GridRestResponse restRes, List<String> params) {
+        // It has to respond with the number of removed entries...
+        return (restRes.getResponse() == null ? GridRedisProtocolParser.toInteger("0")
+            : GridRedisProtocolParser.toInteger(String.valueOf(params.size())));
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b8a2b8ed/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/key/GridRedisExistsCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/key/GridRedisExistsCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/key/GridRedisExistsCommandHandler.java
new file mode 100644
index 0000000..7d0d457
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/key/GridRedisExistsCommandHandler.java
@@ -0,0 +1,88 @@
+/*
+ * 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.rest.handlers.redis.key;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.rest.GridRestProtocolHandler;
+import org.apache.ignite.internal.processors.rest.GridRestResponse;
+import org.apache.ignite.internal.processors.rest.handlers.redis.GridRedisRestCommandHandler;
+import org.apache.ignite.internal.processors.rest.handlers.redis.exception.GridRedisGenericException;
+import org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisCommand;
+import org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisMessage;
+import org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisProtocolParser;
+import org.apache.ignite.internal.processors.rest.request.GridRestCacheRequest;
+import org.apache.ignite.internal.processors.rest.request.GridRestRequest;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import static org.apache.ignite.internal.processors.rest.GridRestCommand.CACHE_GET_ALL;
+import static org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisCommand.EXISTS;
+
+/**
+ * Redis EXISTS command handler.
+ */
+public class GridRedisExistsCommandHandler extends GridRedisRestCommandHandler {
+    /** Supported commands. */
+    private static final Collection<GridRedisCommand> SUPPORTED_COMMANDS = U.sealList(
+        EXISTS
+    );
+
+    /** {@inheritDoc} */
+    public GridRedisExistsCommandHandler(final IgniteLogger log, final GridRestProtocolHandler hnd) {
+        super(log, hnd);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<GridRedisCommand> supportedCommands() {
+        return SUPPORTED_COMMANDS;
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridRestRequest asRestRequest(GridRedisMessage msg) throws IgniteCheckedException {
+        assert msg != null;
+
+        if (msg.messageSize() < 2)
+            throw new GridRedisGenericException("Wrong number of arguments");
+
+        GridRestCacheRequest restReq = new GridRestCacheRequest();
+
+        restReq.clientId(msg.clientId());
+        restReq.key(msg.key());
+        restReq.command(CACHE_GET_ALL);
+
+        List<String> keys = msg.auxMKeys();
+        Map<Object, Object> mget = U.newHashMap(keys.size());
+
+        for (String key : keys)
+            mget.put(key, null);
+
+        restReq.values(mget);
+
+        return restReq;
+    }
+
+    /** {@inheritDoc} */
+    @Override public ByteBuffer makeResponse(final GridRestResponse restRes, List<String> params) {
+        return (restRes.getResponse() == null ? GridRedisProtocolParser.toInteger("0")
+            : GridRedisProtocolParser.toInteger(((Map<Object, Object>)restRes.getResponse()).size()));
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b8a2b8ed/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/server/GridRedisDbSizeCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/server/GridRedisDbSizeCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/server/GridRedisDbSizeCommandHandler.java
new file mode 100644
index 0000000..56d5e28
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/server/GridRedisDbSizeCommandHandler.java
@@ -0,0 +1,75 @@
+/*
+ * 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.rest.handlers.redis.server;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.rest.GridRestProtocolHandler;
+import org.apache.ignite.internal.processors.rest.GridRestResponse;
+import org.apache.ignite.internal.processors.rest.handlers.redis.GridRedisRestCommandHandler;
+import org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisCommand;
+import org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisMessage;
+import org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisProtocolParser;
+import org.apache.ignite.internal.processors.rest.request.GridRestCacheRequest;
+import org.apache.ignite.internal.processors.rest.request.GridRestRequest;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import static org.apache.ignite.internal.processors.rest.GridRestCommand.CACHE_SIZE;
+import static org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisCommand.DBSIZE;
+
+/**
+ * Redis DBSIZE command handler.
+ */
+public class GridRedisDbSizeCommandHandler extends GridRedisRestCommandHandler {
+    /** Supported commands. */
+    private static final Collection<GridRedisCommand> SUPPORTED_COMMANDS = U.sealList(
+        DBSIZE
+    );
+
+    /** {@inheritDoc} */
+    public GridRedisDbSizeCommandHandler(final IgniteLogger log, final GridRestProtocolHandler hnd) {
+        super(log, hnd);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<GridRedisCommand> supportedCommands() {
+        return SUPPORTED_COMMANDS;
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridRestRequest asRestRequest(GridRedisMessage msg) throws IgniteCheckedException {
+        assert msg != null;
+
+        GridRestCacheRequest restReq = new GridRestCacheRequest();
+
+        restReq.clientId(msg.clientId());
+        restReq.key(msg.key());
+        restReq.command(CACHE_SIZE);
+
+        return restReq;
+    }
+
+    /** {@inheritDoc} */
+    @Override public ByteBuffer makeResponse(final GridRestResponse restRes, List<String> params) {
+        return (restRes.getResponse() == null ? GridRedisProtocolParser.toInteger("0")
+            : GridRedisProtocolParser.toInteger(String.valueOf(restRes.getResponse())));
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b8a2b8ed/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisAppendCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisAppendCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisAppendCommandHandler.java
new file mode 100644
index 0000000..60d25fd
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisAppendCommandHandler.java
@@ -0,0 +1,112 @@
+/*
+ * 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.rest.handlers.redis.string;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.rest.GridRestProtocolHandler;
+import org.apache.ignite.internal.processors.rest.GridRestResponse;
+import org.apache.ignite.internal.processors.rest.handlers.redis.GridRedisRestCommandHandler;
+import org.apache.ignite.internal.processors.rest.handlers.redis.exception.GridRedisGenericException;
+import org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisCommand;
+import org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisMessage;
+import org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisProtocolParser;
+import org.apache.ignite.internal.processors.rest.request.GridRestCacheRequest;
+import org.apache.ignite.internal.processors.rest.request.GridRestRequest;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import static org.apache.ignite.internal.processors.rest.GridRestCommand.CACHE_APPEND;
+import static org.apache.ignite.internal.processors.rest.GridRestCommand.CACHE_GET;
+import static org.apache.ignite.internal.processors.rest.GridRestCommand.CACHE_PUT;
+import static org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisCommand.APPEND;
+
+/**
+ * Redis APPEND command handler.
+ */
+public class GridRedisAppendCommandHandler extends GridRedisRestCommandHandler {
+    /** Supported commands. */
+    private static final Collection<GridRedisCommand> SUPPORTED_COMMANDS = U.sealList(
+        APPEND
+    );
+
+    /** Position of the value. */
+    private static final int VAL_POS = 2;
+
+    /** {@inheritDoc} */
+    public GridRedisAppendCommandHandler(final IgniteLogger log, final GridRestProtocolHandler hnd) {
+        super(log, hnd);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<GridRedisCommand> supportedCommands() {
+        return SUPPORTED_COMMANDS;
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridRestRequest asRestRequest(GridRedisMessage msg) throws IgniteCheckedException {
+        assert msg != null;
+
+        if (msg.messageSize() < 3)
+            throw new GridRedisGenericException("Wrong syntax");
+
+        GridRestCacheRequest appendReq = new GridRestCacheRequest();
+        GridRestCacheRequest getReq = new GridRestCacheRequest();
+
+        String val = msg.aux(VAL_POS);
+
+        appendReq.clientId(msg.clientId());
+        appendReq.key(msg.key());
+        appendReq.value(val);
+        appendReq.command(CACHE_APPEND);
+
+        Object resp = hnd.handle(appendReq).getResponse();
+        if (resp != null && !(boolean)resp) {
+            // append on non-existing key.
+            GridRestCacheRequest setReq = new GridRestCacheRequest();
+
+            setReq.clientId(msg.clientId());
+            setReq.key(msg.key());
+            setReq.value(val);
+            setReq.command(CACHE_PUT);
+
+            hnd.handle(setReq);
+        }
+
+        getReq.clientId(msg.clientId());
+        getReq.key(msg.key());
+        getReq.command(CACHE_GET);
+
+        return getReq;
+    }
+
+    /** {@inheritDoc} */
+    @Override public ByteBuffer makeResponse(final GridRestResponse restRes, List<String> params) {
+        if (restRes.getResponse() == null)
+            return GridRedisProtocolParser.nil();
+
+        if (restRes.getResponse() instanceof String) {
+            int resLen = ((String)restRes.getResponse()).length();
+            return GridRedisProtocolParser.toInteger(String.valueOf(resLen));
+        }
+        else
+            return GridRedisProtocolParser.toTypeError("Operation against a key holding the wrong kind of value");
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b8a2b8ed/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisGetCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisGetCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisGetCommandHandler.java
new file mode 100644
index 0000000..7be53b2
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisGetCommandHandler.java
@@ -0,0 +1,81 @@
+/*
+ * 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.rest.handlers.redis.string;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.rest.GridRestProtocolHandler;
+import org.apache.ignite.internal.processors.rest.GridRestResponse;
+import org.apache.ignite.internal.processors.rest.handlers.redis.GridRedisRestCommandHandler;
+import org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisCommand;
+import org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisMessage;
+import org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisProtocolParser;
+import org.apache.ignite.internal.processors.rest.request.GridRestCacheRequest;
+import org.apache.ignite.internal.processors.rest.request.GridRestRequest;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import static org.apache.ignite.internal.processors.rest.GridRestCommand.CACHE_GET;
+import static org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisCommand.GET;
+
+/**
+ * Redis GET command handler.
+ */
+public class GridRedisGetCommandHandler extends GridRedisRestCommandHandler {
+    /** Supported commands. */
+    private static final Collection<GridRedisCommand> SUPPORTED_COMMANDS = U.sealList(
+        GET
+    );
+
+    /** {@inheritDoc} */
+    public GridRedisGetCommandHandler(final IgniteLogger log, final GridRestProtocolHandler hnd) {
+        super(log, hnd);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<GridRedisCommand> supportedCommands() {
+        return SUPPORTED_COMMANDS;
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridRestRequest asRestRequest(GridRedisMessage msg) throws IgniteCheckedException {
+        assert msg != null;
+
+        GridRestCacheRequest restReq = new GridRestCacheRequest();
+
+        restReq.clientId(msg.clientId());
+        restReq.key(msg.key());
+
+        restReq.command(CACHE_GET);
+
+        return restReq;
+    }
+
+    /** {@inheritDoc} */
+    @Override public ByteBuffer makeResponse(final GridRestResponse restRes, List<String> params) {
+        if (restRes.getResponse() == null)
+            return GridRedisProtocolParser.nil();
+
+        if (restRes.getResponse() instanceof String)
+            return GridRedisProtocolParser.toBulkString(restRes.getResponse());
+        else
+            return GridRedisProtocolParser.toTypeError("Operation against a key holding the wrong kind of value");
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b8a2b8ed/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisGetRangeCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisGetRangeCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisGetRangeCommandHandler.java
new file mode 100644
index 0000000..41b1eac
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisGetRangeCommandHandler.java
@@ -0,0 +1,122 @@
+/*
+ * 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.rest.handlers.redis.string;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.rest.GridRestProtocolHandler;
+import org.apache.ignite.internal.processors.rest.GridRestResponse;
+import org.apache.ignite.internal.processors.rest.handlers.redis.GridRedisRestCommandHandler;
+import org.apache.ignite.internal.processors.rest.handlers.redis.exception.GridRedisGenericException;
+import org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisCommand;
+import org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisMessage;
+import org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisProtocolParser;
+import org.apache.ignite.internal.processors.rest.request.GridRestCacheRequest;
+import org.apache.ignite.internal.processors.rest.request.GridRestRequest;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import static org.apache.ignite.internal.processors.rest.GridRestCommand.CACHE_GET;
+import static org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisCommand.GETRANGE;
+
+/**
+ * Redis SETRANGE command handler.
+ */
+public class GridRedisGetRangeCommandHandler extends GridRedisRestCommandHandler {
+    /** Supported commands. */
+    private static final Collection<GridRedisCommand> SUPPORTED_COMMANDS = U.sealList(
+        GETRANGE
+    );
+
+    /** Start offset position in Redis message parameters. */
+    private static final int START_OFFSET_POS = 1;
+
+    /** End offset position in Redis message parameters. */
+    private static final int END_OFFSET_POS = 2;
+
+    /** {@inheritDoc} */
+    public GridRedisGetRangeCommandHandler(final IgniteLogger log, final GridRestProtocolHandler hnd) {
+        super(log, hnd);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<GridRedisCommand> supportedCommands() {
+        return SUPPORTED_COMMANDS;
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridRestRequest asRestRequest(GridRedisMessage msg) throws IgniteCheckedException {
+        assert msg != null;
+
+        if (msg.messageSize() < 4)
+            throw new GridRedisGenericException("Wrong number of arguments");
+
+        GridRestCacheRequest getReq = new GridRestCacheRequest();
+
+        getReq.clientId(msg.clientId());
+        getReq.key(msg.key());
+        getReq.command(CACHE_GET);
+
+        return getReq;
+    }
+
+    /** {@inheritDoc} */
+    @Override public ByteBuffer makeResponse(final GridRestResponse restRes, List<String> params) {
+        if (restRes.getResponse() == null)
+            return GridRedisProtocolParser.toBulkString("");
+
+        if (restRes.getResponse() instanceof String) {
+            String res = String.valueOf(restRes.getResponse());
+            int startOffset;
+            int endOffset;
+
+            try {
+                startOffset = boundedStartOffset(Integer.parseInt(params.get(START_OFFSET_POS)), res.length());
+                endOffset = boundedEndOffset(Integer.parseInt(params.get(END_OFFSET_POS)), res.length());
+            }
+            catch (NumberFormatException e) {
+                U.error(log, "Erroneous offset", e);
+                return GridRedisProtocolParser.toGenericError("Offset is not an integer");
+            }
+
+            return GridRedisProtocolParser.toBulkString(res.substring(startOffset, endOffset));
+        }
+        else
+            return GridRedisProtocolParser.toTypeError("Operation against a key holding the wrong kind of value");
+    }
+
+    /**
+     * @param idx Index.
+     * @param size Bounds.
+     * @return Offset within the bounds.
+     */
+    private int boundedStartOffset(int idx, int size) {
+        return idx >= 0 ? Math.min(idx, size) : size + idx;
+    }
+
+    /**
+     * @param idx Index.
+     * @param size Bounds.
+     * @return Offset within the bounds.
+     */
+    private int boundedEndOffset(int idx, int size) {
+        return idx >= 0 ? Math.min(idx + 1, size) : size + idx + 1;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b8a2b8ed/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisGetSetCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisGetSetCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisGetSetCommandHandler.java
new file mode 100644
index 0000000..781b8ab
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisGetSetCommandHandler.java
@@ -0,0 +1,89 @@
+/*
+ * 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.rest.handlers.redis.string;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.rest.GridRestProtocolHandler;
+import org.apache.ignite.internal.processors.rest.GridRestResponse;
+import org.apache.ignite.internal.processors.rest.handlers.redis.GridRedisRestCommandHandler;
+import org.apache.ignite.internal.processors.rest.handlers.redis.exception.GridRedisGenericException;
+import org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisCommand;
+import org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisMessage;
+import org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisProtocolParser;
+import org.apache.ignite.internal.processors.rest.request.GridRestCacheRequest;
+import org.apache.ignite.internal.processors.rest.request.GridRestRequest;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import static org.apache.ignite.internal.processors.rest.GridRestCommand.CACHE_GET_AND_PUT;
+import static org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisCommand.GETSET;
+
+/**
+ * Redis GETSET command handler.
+ */
+public class GridRedisGetSetCommandHandler extends GridRedisRestCommandHandler {
+    /** Supported commands. */
+    private static final Collection<GridRedisCommand> SUPPORTED_COMMANDS = U.sealList(
+        GETSET
+    );
+
+    /** Value position in Redis message. */
+    private static final int VAL_POS = 2;
+
+    /** {@inheritDoc} */
+    public GridRedisGetSetCommandHandler(final IgniteLogger log, final GridRestProtocolHandler hnd) {
+        super(log, hnd);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<GridRedisCommand> supportedCommands() {
+        return SUPPORTED_COMMANDS;
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridRestRequest asRestRequest(GridRedisMessage msg) throws IgniteCheckedException {
+        assert msg != null;
+
+        if (msg.messageSize() < 3)
+            throw new GridRedisGenericException("Wrong syntax");
+
+        GridRestCacheRequest restReq = new GridRestCacheRequest();
+
+        restReq.clientId(msg.clientId());
+        restReq.key(msg.key());
+        restReq.value(msg.aux(VAL_POS));
+
+        restReq.command(CACHE_GET_AND_PUT);
+
+        return restReq;
+    }
+
+    /** {@inheritDoc} */
+    @Override public ByteBuffer makeResponse(final GridRestResponse restRes, List<String> params) {
+        if (restRes.getResponse() == null)
+            return GridRedisProtocolParser.nil();
+
+        if (restRes.getResponse() instanceof String)
+            return GridRedisProtocolParser.toBulkString(restRes.getResponse());
+        else
+            return GridRedisProtocolParser.toTypeError("Operation against a key holding the wrong kind of value");
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b8a2b8ed/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisIncrDecrCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisIncrDecrCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisIncrDecrCommandHandler.java
new file mode 100644
index 0000000..219768b
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisIncrDecrCommandHandler.java
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.rest.handlers.redis.string;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.rest.GridRestProtocolHandler;
+import org.apache.ignite.internal.processors.rest.GridRestResponse;
+import org.apache.ignite.internal.processors.rest.handlers.redis.GridRedisRestCommandHandler;
+import org.apache.ignite.internal.processors.rest.handlers.redis.exception.GridRedisGenericException;
+import org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisCommand;
+import org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisMessage;
+import org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisProtocolParser;
+import org.apache.ignite.internal.processors.rest.request.DataStructuresRequest;
+import org.apache.ignite.internal.processors.rest.request.GridRestCacheRequest;
+import org.apache.ignite.internal.processors.rest.request.GridRestRequest;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import static org.apache.ignite.internal.processors.rest.GridRestCommand.ATOMIC_DECREMENT;
+import static org.apache.ignite.internal.processors.rest.GridRestCommand.ATOMIC_INCREMENT;
+import static org.apache.ignite.internal.processors.rest.GridRestCommand.CACHE_GET;
+import static org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisCommand.DECR;
+import static org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisCommand.DECRBY;
+import static org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisCommand.INCR;
+import static org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisCommand.INCRBY;
+
+/**
+ * Redis INCR/DECR command handler.
+ */
+public class GridRedisIncrDecrCommandHandler extends GridRedisRestCommandHandler {
+    /** Supported commands. */
+    private static final Collection<GridRedisCommand> SUPPORTED_COMMANDS = U.sealList(
+        INCR,
+        DECR,
+        INCRBY,
+        DECRBY
+    );
+
+    /** Delta position in the message. */
+    private static final int DELTA_POS = 2;
+
+    /** {@inheritDoc} */
+    public GridRedisIncrDecrCommandHandler(final IgniteLogger log, final GridRestProtocolHandler hnd) {
+        super(log, hnd);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<GridRedisCommand> supportedCommands() {
+        return SUPPORTED_COMMANDS;
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridRestRequest asRestRequest(GridRedisMessage msg) throws IgniteCheckedException {
+        assert msg != null;
+
+        DataStructuresRequest restReq = new DataStructuresRequest();
+
+        GridRestCacheRequest getReq = new GridRestCacheRequest();
+
+        getReq.clientId(msg.clientId());
+        getReq.key(msg.key());
+        getReq.command(CACHE_GET);
+
+        GridRestResponse getResp = hnd.handle(getReq);
+
+        if (getResp.getResponse() == null)
+            restReq.initial(0L);
+        else {
+            if (getResp.getResponse() instanceof Long && (Long)getResp.getResponse() <= Long.MAX_VALUE)
+                restReq.initial((Long)getResp.getResponse());
+            else
+                throw new GridRedisGenericException("An initial value must be numeric and in range");
+        }
+
+        restReq.clientId(msg.clientId());
+        restReq.key(msg.key());
+        restReq.delta(1L);
+
+        if (msg.messageSize() > 2) {
+            try {
+                restReq.delta(Long.valueOf(msg.aux(DELTA_POS)));
+            }
+            catch (NumberFormatException e) {
+                U.error(log, "Wrong increment delta", e);
+                throw new GridRedisGenericException("An increment value must be numeric and in range");
+            }
+        }
+
+        switch (msg.command()) {
+            case INCR:
+            case INCRBY:
+                restReq.command(ATOMIC_INCREMENT);
+                break;
+
+            case DECR:
+            case DECRBY:
+                restReq.command(ATOMIC_DECREMENT);
+                break;
+            default:
+                assert false : "Unexpected command received";
+        }
+
+        return restReq;
+    }
+
+    /** {@inheritDoc} */
+    @Override public ByteBuffer makeResponse(final GridRestResponse restRes, List<String> params) {
+        if (restRes.getResponse() == null)
+            return GridRedisProtocolParser.toGenericError("Failed to increment");
+
+        if (restRes.getResponse() instanceof Long && (Long)restRes.getResponse() <= Long.MAX_VALUE)
+            return GridRedisProtocolParser.toInteger(String.valueOf(restRes.getResponse()));
+        else
+            return GridRedisProtocolParser.toTypeError("Value is non-numeric or out of range");
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b8a2b8ed/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisMGetCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisMGetCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisMGetCommandHandler.java
new file mode 100644
index 0000000..e5406c6
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisMGetCommandHandler.java
@@ -0,0 +1,88 @@
+/*
+ * 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.rest.handlers.redis.string;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.rest.GridRestProtocolHandler;
+import org.apache.ignite.internal.processors.rest.GridRestResponse;
+import org.apache.ignite.internal.processors.rest.handlers.redis.GridRedisRestCommandHandler;
+import org.apache.ignite.internal.processors.rest.handlers.redis.exception.GridRedisGenericException;
+import org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisCommand;
+import org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisMessage;
+import org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisProtocolParser;
+import org.apache.ignite.internal.processors.rest.request.GridRestCacheRequest;
+import org.apache.ignite.internal.processors.rest.request.GridRestRequest;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import static org.apache.ignite.internal.processors.rest.GridRestCommand.CACHE_GET_ALL;
+import static org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisCommand.MGET;
+
+/**
+ * Redis MGET command handler.
+ */
+public class GridRedisMGetCommandHandler extends GridRedisRestCommandHandler {
+    /** Supported commands. */
+    private static final Collection<GridRedisCommand> SUPPORTED_COMMANDS = U.sealList(
+        MGET
+    );
+
+    /** {@inheritDoc} */
+    public GridRedisMGetCommandHandler(final IgniteLogger log, final GridRestProtocolHandler hnd) {
+        super(log, hnd);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<GridRedisCommand> supportedCommands() {
+        return SUPPORTED_COMMANDS;
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridRestRequest asRestRequest(GridRedisMessage msg) throws IgniteCheckedException {
+        assert msg != null;
+
+        if (msg.messageSize() < 2)
+            throw new GridRedisGenericException("Wrong number of arguments");
+
+        GridRestCacheRequest restReq = new GridRestCacheRequest();
+
+        restReq.clientId(msg.clientId());
+        restReq.key(msg.key());
+        restReq.command(CACHE_GET_ALL);
+
+        List<String> keys = msg.auxMKeys();
+        Map<Object, Object> mget = U.newHashMap(keys.size());
+
+        for (String key : keys)
+            mget.put(key, null);
+
+        restReq.values(mget);
+
+        return restReq;
+    }
+
+    /** {@inheritDoc} */
+    @Override public ByteBuffer makeResponse(final GridRestResponse restRes, List<String> params) {
+        return (restRes.getResponse() == null ? GridRedisProtocolParser.nil()
+            : GridRedisProtocolParser.toArray((Map<Object, Object>)restRes.getResponse()));
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b8a2b8ed/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisMSetCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisMSetCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisMSetCommandHandler.java
new file mode 100644
index 0000000..3e9076b
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/redis/string/GridRedisMSetCommandHandler.java
@@ -0,0 +1,86 @@
+/*
+ * 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.rest.handlers.redis.string;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.rest.GridRestProtocolHandler;
+import org.apache.ignite.internal.processors.rest.GridRestResponse;
+import org.apache.ignite.internal.processors.rest.handlers.redis.GridRedisRestCommandHandler;
+import org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisCommand;
+import org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisMessage;
+import org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisProtocolParser;
+import org.apache.ignite.internal.processors.rest.request.GridRestCacheRequest;
+import org.apache.ignite.internal.processors.rest.request.GridRestRequest;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import static org.apache.ignite.internal.processors.rest.GridRestCommand.CACHE_PUT_ALL;
+import static org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisCommand.MSET;
+
+/**
+ * Redis MSET command handler.
+ */
+public class GridRedisMSetCommandHandler extends GridRedisRestCommandHandler {
+    /** Supported commands. */
+    private static final Collection<GridRedisCommand> SUPPORTED_COMMANDS = U.sealList(
+        MSET
+    );
+
+    /** {@inheritDoc} */
+    public GridRedisMSetCommandHandler(final IgniteLogger log, final GridRestProtocolHandler hnd) {
+        super(log, hnd);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<GridRedisCommand> supportedCommands() {
+        return SUPPORTED_COMMANDS;
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridRestRequest asRestRequest(GridRedisMessage msg) throws IgniteCheckedException {
+        assert msg != null;
+
+        GridRestCacheRequest restReq = new GridRestCacheRequest();
+
+        restReq.clientId(msg.clientId());
+        restReq.key(msg.key());
+
+        restReq.command(CACHE_PUT_ALL);
+
+        List<String> els = msg.auxMKeys();
+        Map<Object, Object> mset = U.newHashMap(els.size() / 2);
+        Iterator<String> msetIt = els.iterator();
+
+        while (msetIt.hasNext())
+            mset.put(msetIt.next(), msetIt.hasNext() ? msetIt.next() : null);
+
+        restReq.values(mset);
+
+        return restReq;
+    }
+
+    /** {@inheritDoc} */
+    @Override public ByteBuffer makeResponse(final GridRestResponse restRes, List<String> params) {
+        return GridRedisProtocolParser.oKString();
+    }
+}