You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by iv...@apache.org on 2015/07/31 13:51:37 UTC

incubator-ignite git commit: #ignite-1170: node.js: implement scan query support

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-961 eccf050b9 -> c4edc5bab


#ignite-1170: node.js: implement scan query support


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

Branch: refs/heads/ignite-961
Commit: c4edc5bab3512e8eb2c571fcd3ce170593217c26
Parents: eccf050
Author: ivasilinets <iv...@gridgain.com>
Authored: Fri Jul 31 14:49:45 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Fri Jul 31 14:49:45 2015 +0300

----------------------------------------------------------------------
 .../JettyRestProcessorAbstractSelfTest.java     |  73 +++++++-
 .../processors/rest/GridRestCommand.java        |   3 +
 .../processors/rest/GridRestProcessor.java      |   3 +-
 .../handlers/query/QueryCommandHandler.java     | 106 +++++++++---
 .../rest/request/RestQueryRequest.java          | 173 +++++++++++++++++++
 .../rest/request/RestSqlQueryRequest.java       | 125 --------------
 modules/nodejs/src/main/js/apache-ignite.js     |   3 +-
 modules/nodejs/src/main/js/cache.js             |  26 ++-
 modules/nodejs/src/main/js/query.js             |  50 ++++++
 modules/nodejs/src/main/js/scan-query.js        |  50 ++++++
 modules/nodejs/src/main/js/sql-fields-query.js  |  68 +-------
 .../ignite/internal/NodeJsSqlQuerySelfTest.java |   9 +
 modules/nodejs/src/test/js/test-query.js        |  49 +++++-
 .../http/jetty/GridJettyRestHandler.java        |  45 ++++-
 14 files changed, 543 insertions(+), 240 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c4edc5ba/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java
index 5f3d15e..aa838bb 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java
@@ -27,6 +27,7 @@ import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.processors.json.*;
 import org.apache.ignite.internal.processors.rest.handlers.*;
 import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.lang.*;
 import org.apache.ignite.testframework.*;
 
 import java.io.*;
@@ -1296,7 +1297,7 @@ public abstract class JettyRestProcessorAbstractSelfTest extends AbstractRestPro
         Map<String, String> params = new HashMap<>();
         params.put("cmd", GridRestCommand.EXECUTE_SQL_QUERY.key());
         params.put("type", "Person");
-        params.put("psz", "10");
+        params.put("pageSize", "10");
         params.put("cacheName", "person");
         params.put("qry", URLEncoder.encode(qry));
         params.put("arg1", "1000");
@@ -1319,6 +1320,52 @@ public abstract class JettyRestProcessorAbstractSelfTest extends AbstractRestPro
     /**
      * @throws Exception If failed.
      */
+    public void testQueryScan() throws Exception {
+        Map<String, String> params = new HashMap<>();
+        params.put("cmd", GridRestCommand.EXECUTE_SCAN_QUERY.key());
+        params.put("pageSize", "10");
+        params.put("cacheName", "person");
+        params.put("classname", ScanFilter.class.getName());
+
+        String ret = content(params);
+
+        assertNotNull(ret);
+        assertTrue(!ret.isEmpty());
+
+        JSONObject json = JSONObject.fromObject(ret);
+
+        List items = (List)((Map)json.get("response")).get("items");
+
+        assertEquals(2, items.size());
+
+        assertFalse(queryCursorFound());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testIncorrectQueryScan() throws Exception {
+        Map<String, String> params = new HashMap<>();
+        params.put("cmd", GridRestCommand.EXECUTE_SCAN_QUERY.key());
+        params.put("pageSize", "10");
+        params.put("cacheName", "person");
+        params.put("classname", ScanFilter.class.getName() + 1);
+
+        String ret = content(params);
+
+        assertNotNull(ret);
+        assertTrue(!ret.isEmpty());
+
+        JSONObject json = JSONObject.fromObject(ret);
+
+        String err = (String)json.get("error");
+
+        assertTrue(err.contains("Failed to find target class"));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testQuery() throws Exception {
         grid(0).cache(null).put("1", "1");
         grid(0).cache(null).put("2", "2");
@@ -1327,7 +1374,7 @@ public abstract class JettyRestProcessorAbstractSelfTest extends AbstractRestPro
         Map<String, String> params = new HashMap<>();
         params.put("cmd", GridRestCommand.EXECUTE_SQL_QUERY.key());
         params.put("type", "String");
-        params.put("psz", "1");
+        params.put("pageSize", "1");
         params.put("qry", URLEncoder.encode("select * from String"));
 
         String ret = content(params);
@@ -1342,7 +1389,7 @@ public abstract class JettyRestProcessorAbstractSelfTest extends AbstractRestPro
         assertNotNull(qryId);
 
         ret = content(F.asMap("cmd", GridRestCommand.FETCH_SQL_QUERY.key(),
-            "psz", "1", "qryId", String.valueOf(qryId)));
+            "pageSize", "1", "qryId", String.valueOf(qryId)));
 
         json = JSONObject.fromObject(ret);
 
@@ -1354,7 +1401,7 @@ public abstract class JettyRestProcessorAbstractSelfTest extends AbstractRestPro
         assertFalse(last);
 
         ret = content(F.asMap("cmd", GridRestCommand.FETCH_SQL_QUERY.key(),
-            "psz", "1", "qryId", String.valueOf(qryId)));
+            "pageSize", "1", "qryId", String.valueOf(qryId)));
 
         json = JSONObject.fromObject(ret);
 
@@ -1376,7 +1423,7 @@ public abstract class JettyRestProcessorAbstractSelfTest extends AbstractRestPro
 
         Map<String, String> params = new HashMap<>();
         params.put("cmd", GridRestCommand.EXECUTE_SQL_FIELDS_QUERY.key());
-        params.put("psz", "10");
+        params.put("pageSize", "10");
         params.put("cacheName", "person");
         params.put("qry", URLEncoder.encode(qry));
 
@@ -1402,7 +1449,7 @@ public abstract class JettyRestProcessorAbstractSelfTest extends AbstractRestPro
 
         Map<String, String> params = new HashMap<>();
         params.put("cmd", GridRestCommand.EXECUTE_SQL_FIELDS_QUERY.key());
-        params.put("psz", "10");
+        params.put("pageSize", "10");
         params.put("cacheName", "person");
         params.put("qry", URLEncoder.encode(qry));
 
@@ -1440,7 +1487,7 @@ public abstract class JettyRestProcessorAbstractSelfTest extends AbstractRestPro
         Map<String, String> params = new HashMap<>();
         params.put("cmd", GridRestCommand.EXECUTE_SQL_QUERY.key());
         params.put("type", "Person");
-        params.put("psz", "1");
+        params.put("pageSize", "1");
         params.put("cacheName", "person");
         params.put("qry", URLEncoder.encode(qry));
         params.put("arg1", "1000");
@@ -1634,7 +1681,7 @@ public abstract class JettyRestProcessorAbstractSelfTest extends AbstractRestPro
         String qry = "salary > ? and salary <= ?";
 
         String ret = makePostRequest(F.asMap("cmd", GridRestCommand.EXECUTE_SQL_QUERY.key(),
-                "type", "Person", "psz", "10", "cacheName", "person",
+                "type", "Person", "pageSize", "10", "cacheName", "person",
                 "qry", URLEncoder.encode(qry)), "{\"arg\": [1000, 2000]}");
 
         assertNotNull(ret);
@@ -1761,4 +1808,14 @@ public abstract class JettyRestProcessorAbstractSelfTest extends AbstractRestPro
             return id;
         }
     }
+
+    /**
+     * Test filter for scan query.
+     */
+    public static class ScanFilter implements IgniteBiPredicate<Integer, Person> {
+        /** {@inheritDoc} */
+        @Override public boolean apply(Integer integer, Person person) {
+            return person.salary > 1000;
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c4edc5ba/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestCommand.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestCommand.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestCommand.java
index 44dec65..42407cc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestCommand.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestCommand.java
@@ -150,6 +150,9 @@ public enum GridRestCommand {
     /** Execute sql fields query. */
     EXECUTE_SQL_FIELDS_QUERY("qryfldexe"),
 
+    /** Execute scan query. */
+    EXECUTE_SCAN_QUERY("qryscanexe"),
+
     /** Fetch query results. */
     FETCH_SQL_QUERY("qryfetch"),
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c4edc5ba/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 f8ccf8b..8871d1a 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
@@ -540,10 +540,11 @@ public class GridRestProcessor extends GridProcessorAdapter {
 
             case EXECUTE_SQL_QUERY:
             case EXECUTE_SQL_FIELDS_QUERY:
+            case EXECUTE_SCAN_QUERY:
             case CLOSE_SQL_QUERY:
             case FETCH_SQL_QUERY:
                 perm = SecurityPermission.CACHE_READ;
-                name = ((RestSqlQueryRequest)req).cacheName();
+                name = ((RestQueryRequest)req).cacheName();
 
                 break;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c4edc5ba/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/query/QueryCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/query/QueryCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/query/QueryCommandHandler.java
index 35fbcef..e121b23 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/query/QueryCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/query/QueryCommandHandler.java
@@ -29,6 +29,7 @@ import org.apache.ignite.internal.util.future.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.lang.*;
 
+import java.lang.reflect.*;
 import java.util.*;
 import java.util.concurrent.*;
 import java.util.concurrent.atomic.*;
@@ -42,6 +43,7 @@ public class QueryCommandHandler extends GridRestCommandHandlerAdapter {
     /** Supported commands. */
     private static final Collection<GridRestCommand> SUPPORTED_COMMANDS = U.sealList(EXECUTE_SQL_QUERY,
         EXECUTE_SQL_FIELDS_QUERY,
+        EXECUTE_SCAN_QUERY,
         FETCH_SQL_QUERY,
         CLOSE_SQL_QUERY);
 
@@ -68,23 +70,24 @@ public class QueryCommandHandler extends GridRestCommandHandlerAdapter {
         assert req != null;
 
         assert SUPPORTED_COMMANDS.contains(req.command());
-        assert req instanceof RestSqlQueryRequest : "Invalid type of query request.";
+        assert req instanceof RestQueryRequest : "Invalid type of query request.";
 
         switch (req.command()) {
             case EXECUTE_SQL_QUERY:
-            case EXECUTE_SQL_FIELDS_QUERY: {
+            case EXECUTE_SQL_FIELDS_QUERY:
+            case EXECUTE_SCAN_QUERY: {
                 return ctx.closure().callLocalSafe(
-                    new ExecuteQueryCallable(ctx, (RestSqlQueryRequest)req, qryCurs), false);
+                    new ExecuteQueryCallable(ctx, (RestQueryRequest)req, qryCurs), false);
             }
 
             case FETCH_SQL_QUERY: {
                 return ctx.closure().callLocalSafe(
-                    new FetchQueryCallable(ctx, (RestSqlQueryRequest)req, qryCurs), false);
+                    new FetchQueryCallable(ctx, (RestQueryRequest)req, qryCurs), false);
             }
 
             case CLOSE_SQL_QUERY: {
                 return ctx.closure().callLocalSafe(
-                    new CloseQueryCallable((RestSqlQueryRequest)req, qryCurs), false);
+                    new CloseQueryCallable((RestQueryRequest)req, qryCurs), false);
             }
         }
 
@@ -99,7 +102,7 @@ public class QueryCommandHandler extends GridRestCommandHandlerAdapter {
         private GridKernalContext ctx;
 
         /** Execute query request. */
-        private RestSqlQueryRequest req;
+        private RestQueryRequest req;
 
         /** Queries cursors. */
         private ConcurrentHashMap<Long, IgniteBiTuple<QueryCursor, Iterator>> qryCurs;
@@ -109,7 +112,7 @@ public class QueryCommandHandler extends GridRestCommandHandlerAdapter {
          * @param req Execute query request.
          * @param qryCurs Queries cursors.
          */
-        public ExecuteQueryCallable(GridKernalContext ctx, RestSqlQueryRequest req,
+        public ExecuteQueryCallable(GridKernalContext ctx, RestQueryRequest req,
             ConcurrentHashMap<Long, IgniteBiTuple<QueryCursor, Iterator>> qryCurs) {
             this.ctx = ctx;
             this.req = req;
@@ -123,22 +126,40 @@ public class QueryCommandHandler extends GridRestCommandHandlerAdapter {
             try {
                 Query qry;
 
-                if (req.typeName() != null) {
-                    qry = new SqlQuery(req.typeName(), req.sqlQuery());
+                switch (req.queryType()) {
+                    case SQL:
+                        qry = new SqlQuery(req.typeName(), req.sqlQuery());
 
-                    ((SqlQuery)qry).setArgs(req.arguments());
-                }
-                else {
-                    qry = new SqlFieldsQuery(req.sqlQuery());
+                        ((SqlQuery)qry).setArgs(req.arguments());
+
+                        break;
+
+                    case SQL_FIELDS:
+                        qry = new SqlFieldsQuery(req.sqlQuery());
+
+                        ((SqlFieldsQuery)qry).setArgs(req.arguments());
+
+                        break;
+
+                    case SCAN:
+                        IgniteBiPredicate pred = null;
 
-                    ((SqlFieldsQuery)qry).setArgs(req.arguments());
+                        if (req.className() != null)
+                            pred = instance(IgniteBiPredicate.class, req.className());
+
+                        qry = new ScanQuery(pred);
+
+                        break;
+
+                    default:
+                        throw new IgniteException("Incorrect query type [type=" + req.queryType() + "]");
                 }
 
                 IgniteCache<Object, Object> cache = ctx.grid().cache(req.cacheName());
 
                 if (cache == null)
                     return new GridRestResponse(GridRestResponse.STATUS_FAILED,
-                        "No cache with name [cacheName=" + req.cacheName() + "]");
+                        "Failed to find cache with name: " + req.cacheName());
 
                 QueryCursor qryCur = cache.query(qry);
 
@@ -182,7 +203,7 @@ public class QueryCommandHandler extends GridRestCommandHandlerAdapter {
      */
     private static class CloseQueryCallable implements Callable<GridRestResponse> {
         /** Execute query request. */
-        private RestSqlQueryRequest req;
+        private RestQueryRequest req;
 
         /** Queries cursors. */
         private final ConcurrentHashMap<Long, IgniteBiTuple<QueryCursor, Iterator>> qryCurs;
@@ -191,7 +212,7 @@ public class QueryCommandHandler extends GridRestCommandHandlerAdapter {
          * @param req Execute query request.
          * @param qryCurs Queries cursors.
          */
-        public CloseQueryCallable(RestSqlQueryRequest req,
+        public CloseQueryCallable(RestQueryRequest req,
             ConcurrentHashMap<Long, IgniteBiTuple<QueryCursor, Iterator>> qryCurs) {
             this.req = req;
             this.qryCurs = qryCurs;
@@ -204,7 +225,7 @@ public class QueryCommandHandler extends GridRestCommandHandlerAdapter {
 
                 if (cur == null)
                     return new GridRestResponse(GridRestResponse.STATUS_FAILED,
-                        "Cannot find query [qryId=" + req.queryId() + "]");
+                        "Failed to find query with ID: " + req.queryId());
 
                 cur.close();
 
@@ -225,7 +246,7 @@ public class QueryCommandHandler extends GridRestCommandHandlerAdapter {
      */
     private static class FetchQueryCallable implements Callable<GridRestResponse> {
         /** Execute query request. */
-        private RestSqlQueryRequest req;
+        private RestQueryRequest req;
 
         /** Queries cursors. */
         private final ConcurrentHashMap<Long, IgniteBiTuple<QueryCursor, Iterator>> qryCurs;
@@ -238,7 +259,7 @@ public class QueryCommandHandler extends GridRestCommandHandlerAdapter {
          * @param req Execute query request.
          * @param qryCurs Queries cursors.
          */
-        public FetchQueryCallable(GridKernalContext ctx, RestSqlQueryRequest req,
+        public FetchQueryCallable(GridKernalContext ctx, RestQueryRequest req,
             ConcurrentHashMap<Long, IgniteBiTuple<QueryCursor, Iterator>> qryCurs) {
             this.ctx = ctx;
             this.req = req;
@@ -252,7 +273,7 @@ public class QueryCommandHandler extends GridRestCommandHandlerAdapter {
 
                 if (cur == null)
                     return new GridRestResponse(GridRestResponse.STATUS_FAILED,
-                        "Cannot find query [qryId=" + req.queryId() + "]");
+                        "Failed to find query with ID: " + req.queryId());
 
                 CacheQueryResult res = createQueryResult(qryCurs, cur, req, req.queryId(), ctx);
 
@@ -276,7 +297,7 @@ public class QueryCommandHandler extends GridRestCommandHandlerAdapter {
      */
     private static CacheQueryResult createQueryResult(
         ConcurrentHashMap<Long, IgniteBiTuple<QueryCursor, Iterator>> qryCurs,
-        Iterator cur, RestSqlQueryRequest req, Long qryId, GridKernalContext ctx) {
+        Iterator cur, RestQueryRequest req, Long qryId, GridKernalContext ctx) {
         CacheQueryResult res = new CacheQueryResult();
 
         List<Object> items = new ArrayList<>();
@@ -295,4 +316,45 @@ public class QueryCommandHandler extends GridRestCommandHandlerAdapter {
 
         return res;
     }
+
+    /**
+     * Creates class instance.
+     *
+     * @param cls Target class.
+     * @param clsName Implementing class name.
+     * @return Class instance.
+     * @throws IgniteException If failed.
+     */
+    private static <T> T instance(Class<? extends T> cls, String clsName) throws IgniteException {
+        try {
+            Class<?> implCls = Class.forName(clsName);
+
+            if (!cls.isAssignableFrom(implCls))
+                throw new IgniteException("Failed to create instance (target class does not extend or implement " +
+                    "required class or interface) [cls=" + cls.getName() + ", clsName=" + clsName + ']');
+
+            Constructor<?> ctor = implCls.getConstructor();
+
+            return (T)ctor.newInstance();
+        }
+        catch (ClassNotFoundException e) {
+            throw new IgniteException("Failed to find target class: " + clsName, e);
+        }
+        catch (NoSuchMethodException e) {
+            throw new IgniteException("Failed to find constructor for provided arguments " +
+                "[clsName=" + clsName + ']', e);
+        }
+        catch (InstantiationException e) {
+            throw new IgniteException("Failed to instantiate target class " +
+                "[clsName=" + clsName + ']', e);
+        }
+        catch (IllegalAccessException e) {
+            throw new IgniteException("Failed to instantiate class (constructor is not available) " +
+                "[clsName=" + clsName + ']', e);
+        }
+        catch (InvocationTargetException e) {
+            throw new IgniteException("Failed to instantiate class (constructor threw an exception) " +
+                "[clsName=" + clsName + ']', e.getCause());
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c4edc5ba/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/RestQueryRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/RestQueryRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/RestQueryRequest.java
new file mode 100644
index 0000000..029b573
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/RestQueryRequest.java
@@ -0,0 +1,173 @@
+/*
+ * 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.request;
+
+/**
+ * Sql query request.
+ */
+public class RestQueryRequest extends GridRestRequest {
+    /** Sql query. */
+    private String sqlQry;
+
+    /** Sql query arguments. */
+    private Object[] args;
+
+    /** Page size. */
+    private Integer pageSize;
+
+    /** Cache name. */
+    private String cacheName;
+
+    /** Query id. */
+    private Long qryId;
+
+    /** Query type name. */
+    private String typeName;
+
+    /** Predicate class name for scan query. */
+    private String className;
+
+    /** Query type. */
+    private QueryType type;
+
+    /**
+     * @param sqlQry Sql query.
+     */
+    public void sqlQuery(String sqlQry) {
+        this.sqlQry = sqlQry;
+    }
+
+    /**
+     * @return Sql query.
+     */
+    public String sqlQuery() {
+        return sqlQry;
+    }
+
+    /**
+     * @param args Sql query arguments.
+     */
+    public void arguments(Object[] args) {
+        this.args = args;
+    }
+
+    /**
+     * @return Sql query arguments.
+     */
+    public Object[] arguments() {
+        return args;
+    }
+
+    /**
+     * @param pageSize Page size.
+     */
+    public void pageSize(Integer pageSize) {
+        this.pageSize = pageSize;
+    }
+
+    /**
+     * @return Page size.
+     */
+    public int pageSize() {
+        return pageSize;
+    }
+
+    /**
+     * @param cacheName Cache name.
+     */
+    public void cacheName(String cacheName) {
+        this.cacheName = cacheName;
+    }
+
+    /**
+     * @return Cache name.
+     */
+    public String cacheName() {
+        return cacheName;
+    }
+
+    /**
+     * @param id Query id.
+     */
+    public void queryId(Long id) {
+        this.qryId = id;
+    }
+
+    /**
+     * @return Query id.
+     */
+    public Long queryId() {
+        return qryId;
+    }
+
+    /**
+     * @param typeName Query type name.
+     */
+    public void typeName(String typeName) {
+        this.typeName = typeName;
+    }
+
+    /**
+     * @return Query type name.
+     */
+    public String typeName() {
+        return typeName;
+    }
+
+    /**
+     * @return Predicate class name for scan query.
+     */
+    public String className() {
+        return className;
+    }
+
+    /**
+     * @param className Predicate class name for scan query.
+     */
+    public void className(String className) {
+        this.className = className;
+    }
+
+    /**
+     * @param type Query type.
+     */
+    public void queryType(QueryType type) {
+        this.type = type;
+    }
+
+    /**
+     * @return Query type.
+     */
+    public QueryType queryType() {
+        return type;
+    }
+
+    /**
+     * Supported query types.
+     */
+    public enum QueryType {
+        /** Sql query. */
+        SQL,
+
+        /** Sql fields query. */
+        SQL_FIELDS,
+
+        /** Scan query. */
+        SCAN
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c4edc5ba/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/RestSqlQueryRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/RestSqlQueryRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/RestSqlQueryRequest.java
deleted file mode 100644
index 5ba3a50..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/RestSqlQueryRequest.java
+++ /dev/null
@@ -1,125 +0,0 @@
-/*
- * 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.request;
-
-/**
- * Sql query request.
- */
-public class RestSqlQueryRequest extends GridRestRequest {
-    /** Sql query. */
-    private String sqlQry;
-
-    /** Sql query arguments. */
-    private Object[] args;
-
-    /** Page size. */
-    private Integer pageSize;
-
-    /** Cache name. */
-    private String cacheName;
-
-    /** Query id. */
-    private Long qryId;
-
-    /** Query type name. */
-    private String typeName;
-
-    /**
-     * @param sqlQry Sql query.
-     */
-    public void sqlQuery(String sqlQry) {
-        this.sqlQry = sqlQry;
-    }
-
-    /**
-     * @return Sql query.
-     */
-    public String sqlQuery() {
-        return sqlQry;
-    }
-
-    /**
-     * @param args Sql query arguments.
-     */
-    public void arguments(Object[] args) {
-        this.args = args;
-    }
-
-    /**
-     * @return Sql query arguments.
-     */
-    public Object[] arguments() {
-        return args;
-    }
-
-    /**
-     * @param pageSize Page size.
-     */
-    public void pageSize(Integer pageSize) {
-        this.pageSize = pageSize;
-    }
-
-    /**
-     * @return Page size.
-     */
-    public int pageSize() {
-        return pageSize;
-    }
-
-    /**
-     * @param cacheName Cache name.
-     */
-    public void cacheName(String cacheName) {
-        this.cacheName = cacheName;
-    }
-
-    /**
-     * @return Cache name.
-     */
-    public String cacheName() {
-        return cacheName;
-    }
-
-    /**
-     * @param id Query id.
-     */
-    public void queryId(Long id) {
-        this.qryId = id;
-    }
-
-    /**
-     * @return Query id.
-     */
-    public Long queryId() {
-        return qryId;
-    }
-
-    /**
-     * @param typeName Query type name.
-     */
-    public void typeName(String typeName) {
-        this.typeName = typeName;
-    }
-
-    /**
-     * @return Query type name.
-     */
-    public String typeName() {
-        return typeName;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c4edc5ba/modules/nodejs/src/main/js/apache-ignite.js
----------------------------------------------------------------------
diff --git a/modules/nodejs/src/main/js/apache-ignite.js b/modules/nodejs/src/main/js/apache-ignite.js
index 053b88a..d4eb659 100644
--- a/modules/nodejs/src/main/js/apache-ignite.js
+++ b/modules/nodejs/src/main/js/apache-ignite.js
@@ -22,5 +22,6 @@ module.exports = {
     Ignite : require('./ignite.js').Ignite,
     Compute : require('./compute.js').Compute,
     SqlQuery : require('./sql-query.js').SqlQuery,
-    SqlFieldsQuery : require('./sql-fields-query.js').SqlFieldsQuery
+    SqlFieldsQuery : require('./sql-fields-query.js').SqlFieldsQuery,
+    ScanQuery : require('./scan-query.js').ScanQuery
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c4edc5ba/modules/nodejs/src/main/js/cache.js
----------------------------------------------------------------------
diff --git a/modules/nodejs/src/main/js/cache.js b/modules/nodejs/src/main/js/cache.js
index 1600eaa..6c79ff0 100644
--- a/modules/nodejs/src/main/js/cache.js
+++ b/modules/nodejs/src/main/js/cache.js
@@ -463,17 +463,23 @@ QueryCursor.prototype.isFinished = function() {
 
 QueryCursor.prototype._getQueryCommand = function() {
     if (this._init) {
+        this._init = false;
+
         if (this._qry.type() === "Sql") {
             return this._sqlQuery(this._qry);
         }
+        else if (this._qry.type() == "SqlFields") {
+            return this._sqlFieldsQuery(this._qry);
+        }
+        else if (this._qry.type() == "Scan") {
+            return this._scanQuery(this._qry);
+        }
 
-        this._init = false;
-
-        return this._sqlFieldsQuery(this._qry);
+        return null;
     }
 
     return this._cache._createCommand("qryfetch").addParam("qryId", this._res.queryId).
-        addParam("psz", this._qry.pageSize());
+        addParam("pageSize", this._qry.pageSize());
 }
 
 QueryCursor.prototype._sqlFieldsQuery = function(qry) {
@@ -486,9 +492,19 @@ QueryCursor.prototype._sqlQuery = function(qry) {
         setPostData(JSON.stringify({"arg" : qry.arguments()}));
 }
 
+QueryCursor.prototype._scanQuery = function(qry) {
+    var cmd = new Command("qryscanexe").addParam("cacheName", this._cache._cacheName).
+        addParam("pageSize", qry.pageSize());
+
+    if (qry.filterClassName() != null)
+        cmd.addParam("classname", qry.filterClassName());
+
+    return cmd;
+}
+
 QueryCursor.prototype._createQueryCommand = function(name, qry) {
     return new Command(name).addParam("cacheName", this._cache._cacheName).
-        addParam("qry", qry.query()).addParam("psz", qry.pageSize());
+        addParam("qry", qry.query()).addParam("pageSize", qry.pageSize());
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c4edc5ba/modules/nodejs/src/main/js/query.js
----------------------------------------------------------------------
diff --git a/modules/nodejs/src/main/js/query.js b/modules/nodejs/src/main/js/query.js
new file mode 100644
index 0000000..576a95d
--- /dev/null
+++ b/modules/nodejs/src/main/js/query.js
@@ -0,0 +1,50 @@
+/*
+ * 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.
+ */
+
+/**
+ * @this {Query}
+ */
+function Query() {
+    this._qryType = "";
+    this._pageSz = 1;
+}
+
+/**
+ * @this {Query}
+ * @param {int} pageSz Page size.
+ */
+Query.prototype.setPageSize = function(pageSz) {
+    this._pageSz = pageSz;
+}
+
+/**
+ * @this {Query}
+ * @returns pageSize
+ */
+Query.prototype.pageSize = function() {
+    return this._pageSz;
+}
+
+/**
+ * @this {Query}
+ * @returns "SqlFields"
+ */
+Query.prototype.type = function() {
+    return this._qryType;
+}
+
+exports.Query = Query;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c4edc5ba/modules/nodejs/src/main/js/scan-query.js
----------------------------------------------------------------------
diff --git a/modules/nodejs/src/main/js/scan-query.js b/modules/nodejs/src/main/js/scan-query.js
new file mode 100644
index 0000000..876919c
--- /dev/null
+++ b/modules/nodejs/src/main/js/scan-query.js
@@ -0,0 +1,50 @@
+/*
+ * 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.
+ */
+
+var Query = require("./query").Query
+
+/**
+ * @this {ScanQuery}
+ */
+function ScanQuery() {
+    Query.apply(this, arguments);
+    this._className = null;
+    this._qryType = "Scan";
+}
+
+ScanQuery.prototype = Query.prototype;
+
+ScanQuery.prototype.constructor = ScanQuery;
+
+
+/**
+ * @this {ScanQuery}
+ * @param type Filter class name
+ */
+ScanQuery.prototype.setFilterClassName = function(className) {
+    this._className = className;
+}
+
+/**
+ * @this {ScanQuery}
+ * @returns Filter class name
+ */
+ScanQuery.prototype.filterClassName = function() {
+    return this._className;
+}
+
+exports.ScanQuery = ScanQuery;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c4edc5ba/modules/nodejs/src/main/js/sql-fields-query.js
----------------------------------------------------------------------
diff --git a/modules/nodejs/src/main/js/sql-fields-query.js b/modules/nodejs/src/main/js/sql-fields-query.js
index edc9f4c..82fbb93 100644
--- a/modules/nodejs/src/main/js/sql-fields-query.js
+++ b/modules/nodejs/src/main/js/sql-fields-query.js
@@ -15,65 +15,23 @@
  * limitations under the License.
  */
 
+var Query = require("./query").Query
+
 /**
  * @this {SqlFieldsQuery}
  * @param {string} Sql query
  */
 function SqlFieldsQuery(sql) {
+    Query.apply(this, arguments);
     this._qryType = "SqlFields";
     this._sql = sql;
     this._arg = [];
     this._pageSz = 1;
-    this._type = null;
-    this._endFunc = function(err) {console.log("Empty end function is called [err=" + err + "]")};
-    this._pageFunc = function(res) {console.log("Empty page function is called [res=" + res + "]")}
-}
-
-/**
- * Set the callbacks for query events.
- *
- * @this {SqlFieldsQuery}
- * @param {string} code Function code could be "end", "page"
- * @param function Functions "end" and "page" are one argument functions.
- */
-SqlFieldsQuery.prototype.on = function(code, f) {
-    switch(code) {
-        case "end":
-            this._endFunc = f;
-
-            break;
-        case "page":
-            this._pageFunc = f;
-
-            break;
-        default :
-            throw "Sql do not have method " + code;
-    }
-}
-
-/**
- * @this {SqlFieldsQuery}
- * @param res Query result
- */
-SqlFieldsQuery.prototype.end = function(err) {
-    this._endFunc(err);
 }
 
-/**
- * @this {SqlFieldsQuery}
- * @param res Query data
- */
-SqlFieldsQuery.prototype.page = function(res) {
-    this._pageFunc(res);
-}
+SqlFieldsQuery.prototype = Query.prototype;
 
-/**
- * @this {SqlFieldsQuery}
- * @param {int} pageSz Page size.
- */
-SqlFieldsQuery.prototype.setPageSize = function(pageSz) {
-    this._pageSz = pageSz;
-}
+SqlFieldsQuery.prototype.constructor = SqlFieldsQuery;
 
 /**
  * @this {SqlFieldsQuery}
@@ -99,20 +57,4 @@ SqlFieldsQuery.prototype.arguments = function() {
     return this._arg;
 }
 
-/**
- * @this {SqlFieldsQuery}
- * @returns pageSize
- */
-SqlFieldsQuery.prototype.pageSize = function() {
-    return this._pageSz;
-}
-
-/**
- * @this {SqlFieldsQuery}
- * @returns "SqlFields"
- */
-SqlFieldsQuery.prototype.type = function() {
-    return this._qryType;
-}
-
 exports.SqlFieldsQuery = SqlFieldsQuery;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c4edc5ba/modules/nodejs/src/test/java/org/apache/ignite/internal/NodeJsSqlQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/nodejs/src/test/java/org/apache/ignite/internal/NodeJsSqlQuerySelfTest.java b/modules/nodejs/src/test/java/org/apache/ignite/internal/NodeJsSqlQuerySelfTest.java
index 22d7ad4..f290820 100644
--- a/modules/nodejs/src/test/java/org/apache/ignite/internal/NodeJsSqlQuerySelfTest.java
+++ b/modules/nodejs/src/test/java/org/apache/ignite/internal/NodeJsSqlQuerySelfTest.java
@@ -99,6 +99,15 @@ public class NodeJsSqlQuerySelfTest extends NodeJsAbstractTest {
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testScanQuery() throws Exception {
+        initCache();
+
+        runJsScript("testScanQuery");
+    }
+
+    /**
      * Init cache.
      */
     private void initCache() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c4edc5ba/modules/nodejs/src/test/js/test-query.js
----------------------------------------------------------------------
diff --git a/modules/nodejs/src/test/js/test-query.js b/modules/nodejs/src/test/js/test-query.js
index 3d55886..cad653c 100644
--- a/modules/nodejs/src/test/js/test-query.js
+++ b/modules/nodejs/src/test/js/test-query.js
@@ -22,6 +22,7 @@ var assert = require("assert");
 var Ignite = require(TestUtils.scriptPath());
 var SqlQuery = Ignite.SqlQuery;
 var SqlFieldsQuery = Ignite.SqlFieldsQuery;
+var ScanQuery = Ignite.ScanQuery;
 
 testSqlQuery = function() {
     TestUtils.startIgniteNode().then(function(ignite) {
@@ -59,10 +60,46 @@ testSqlQuery = function() {
 
             cursor.nextPage().then(onQuery);
         }).catch(function(err) {
-            assert(err === null, err);
+            TestUtils.testFails(err);
         })
     }).catch(function(err) {
-        assert(err === null, err);
+        TestUtils.testFails(err);
+    });
+}
+
+testScanQuery = function() {
+    TestUtils.startIgniteNode().then(function(ignite) {
+        var qry = new ScanQuery();
+
+        var fullRes = [];
+
+        function onQuery(cursor) {
+            var page = cursor.page();
+
+            fullRes = fullRes.concat(page);
+
+            if (cursor.isFinished()) {
+                console.log("Full result=" + JSON.stringify(fullRes));
+
+                assert(fullRes.length === 4, "Result length is not correct" +
+                    "[expected=1, val = " + fullRes.length + "]");
+
+                fullRes.sort();
+
+                assert(fullRes[0]["key"] >= 0,
+                    "Result has incorrect index [res=" + fullRes[0]["key"] + "]");
+
+                return ignite.cache("person").get("key");
+            }
+
+            return cursor.nextPage().then(onQuery);
+        }
+
+        ignite.cache("person").query(qry).nextPage().then(onQuery).then(function(){
+            TestUtils.testDone();
+        })
+    }).catch(function(err) {
+        TestUtils.testFails(err);
     });
 }
 
@@ -98,7 +135,7 @@ testSqlFieldsQuery = function() {
             TestUtils.testDone();
         })
     }).catch(function(err) {
-        assert(err === null, err);
+        TestUtils.testFails(err);
     });
 }
 
@@ -148,7 +185,7 @@ testSqlFieldsGetAllQuery = function() {
             TestUtils.testDone();
         })
     }).catch(function(err) {
-        assert(err === null, err);
+        TestUtils.testFails(err);
     });
 }
 
@@ -193,7 +230,7 @@ testSqlFieldsMeta = function() {
 
         ignite.cache("person").query(qry).nextPage().then(onQuery);
     }).catch(function(err) {
-        assert(err === null, err);
+        TestUtils.testFails(err);
     });
 }
 
@@ -232,6 +269,6 @@ testSqlQueryWithParams = function() {
 
         ignite.cache("person").query(qry).nextPage().then(onQuery);
     }).catch(function(err) {
-        assert(err === null, err);
+        TestUtils.testFails(err);
     });
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c4edc5ba/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 2cf9988..17467ca 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
@@ -24,6 +24,7 @@ import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.rest.*;
 import org.apache.ignite.internal.processors.rest.client.message.*;
 import org.apache.ignite.internal.processors.rest.request.*;
+import org.apache.ignite.internal.processors.rest.request.RestQueryRequest.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.lang.*;
@@ -639,7 +640,7 @@ public class GridJettyRestHandler extends AbstractHandler {
 
             case EXECUTE_SQL_QUERY:
             case EXECUTE_SQL_FIELDS_QUERY: {
-                RestSqlQueryRequest restReq0 = new RestSqlQueryRequest();
+                RestQueryRequest restReq0 = new RestQueryRequest();
 
                 restReq0.sqlQuery((String) params.get("qry"));
 
@@ -653,30 +654,56 @@ public class GridJettyRestHandler extends AbstractHandler {
 
                 restReq0.typeName((String) params.get("type"));
 
-                String psz = (String) params.get("psz");
+                String pageSize = (String) params.get("pageSize");
 
-                if (psz != null)
-                    restReq0.pageSize(Integer.parseInt(psz));
+                if (pageSize != null)
+                    restReq0.pageSize(Integer.parseInt(pageSize));
 
                 restReq0.cacheName((String)params.get("cacheName"));
 
+                if (cmd.equals(EXECUTE_SQL_QUERY))
+                    restReq0.queryType(QueryType.SQL);
+                else
+                    restReq0.queryType(QueryType.SQL_FIELDS);
+
+                restReq = restReq0;
+
+                break;
+            }
+
+            case EXECUTE_SCAN_QUERY: {
+                RestQueryRequest restReq0 = new RestQueryRequest();
+
+                restReq0.sqlQuery((String)params.get("qry"));
+
+                String pageSize = (String)params.get("pageSize");
+
+                if (pageSize != null)
+                    restReq0.pageSize(Integer.parseInt(pageSize));
+
+                restReq0.cacheName((String)params.get("cacheName"));
+
+                restReq0.className((String)params.get("classname"));
+
+                restReq0.queryType(QueryType.SCAN);
+
                 restReq = restReq0;
 
                 break;
             }
 
             case FETCH_SQL_QUERY: {
-                RestSqlQueryRequest restReq0 = new RestSqlQueryRequest();
+                RestQueryRequest restReq0 = new RestQueryRequest();
 
                 String qryId = (String) params.get("qryId");
 
                 if (qryId != null)
                     restReq0.queryId(Long.parseLong(qryId));
 
-                String psz = (String) params.get("psz");
+                String pageSize = (String)params.get("pageSize");
 
-                if (psz != null)
-                    restReq0.pageSize(Integer.parseInt(psz));
+                if (pageSize != null)
+                    restReq0.pageSize(Integer.parseInt(pageSize));
 
                 restReq0.cacheName((String)params.get("cacheName"));
 
@@ -686,7 +713,7 @@ public class GridJettyRestHandler extends AbstractHandler {
             }
 
             case CLOSE_SQL_QUERY: {
-                RestSqlQueryRequest restReq0 = new RestSqlQueryRequest();
+                RestQueryRequest restReq0 = new RestQueryRequest();
 
                 String qryId = (String) params.get("qryId");