You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2015/06/29 19:40:21 UTC

[41/41] incubator-ignite git commit: #ignite-964: wip.

#ignite-964: wip.


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

Branch: refs/heads/ignite-964
Commit: d4a290060021d30a62e5c369c4449bc7546e4956
Parents: 2a245b2
Author: ivasilinets <iv...@gridgain.com>
Authored: Mon Jun 29 20:39:23 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Mon Jun 29 20:39:23 2015 +0300

----------------------------------------------------------------------
 .../processors/rest/GridRestCommand.java        |   3 +
 .../handlers/query/QueryCommandHandler.java     |  13 +-
 modules/nodejs/src/main/js/apache-ignite.js     |   3 +-
 modules/nodejs/src/main/js/cache.js             |  28 +++-
 modules/nodejs/src/main/js/server.js            |   5 +
 modules/nodejs/src/main/js/sql-fields-query.js  | 131 +++++++++++++++++++
 modules/nodejs/src/main/js/sql-query.js         | 114 +---------------
 .../apache/ignite/internal/NodeJsSqlQuery.java  | 118 +++++++++++++++++
 modules/nodejs/src/test/js/test-query.js        |  39 ++++++
 .../http/jetty/GridJettyRestHandler.java        |   3 +-
 10 files changed, 343 insertions(+), 114 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d4a29006/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 3893eea..0afefb6 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
@@ -108,6 +108,9 @@ public enum GridRestCommand {
     /** Execute sql query. */
     EXECUTE_SQL_QUERY("qryexecute"),
 
+    /** Execute sql fields query. */
+    EXECUTE_SQL_FIELDS_QUERY("qryfieldsexecute"),
+
     /** Fetch query results. */
     FETCH_SQL_QUERY("qryfetch");
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d4a29006/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 60efbd2..b550a46 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
@@ -38,6 +38,7 @@ import static org.apache.ignite.internal.processors.rest.GridRestCommand.*;
 public class QueryCommandHandler extends GridRestCommandHandlerAdapter {
     /** Supported commands. */
     private static final Collection<GridRestCommand> SUPPORTED_COMMANDS = U.sealList(EXECUTE_SQL_QUERY,
+        EXECUTE_SQL_FIELDS_QUERY,
         FETCH_SQL_QUERY);
 
     /** Query ID sequence. */
@@ -66,7 +67,8 @@ public class QueryCommandHandler extends GridRestCommandHandlerAdapter {
         assert SUPPORTED_COMMANDS.contains(req.command());
 
         switch (req.command()) {
-            case EXECUTE_SQL_QUERY: {
+            case EXECUTE_SQL_QUERY:
+            case EXECUTE_SQL_FIELDS_QUERY: {
                 assert req instanceof RestSqlQueryRequest : "Invalid type of query request.";
 
                 return ctx.closure().callLocalSafe(
@@ -114,7 +116,12 @@ public class QueryCommandHandler extends GridRestCommandHandlerAdapter {
         /** {@inheritDoc} */
         @Override public GridRestResponse call() throws Exception {
             try {
-                SqlQuery<String, String> qry = new SqlQuery(req.typeName(), req.sqlQuery());
+                Query qry;
+
+                if (req.typeName() != null)
+                    qry = new SqlQuery(req.typeName(), req.sqlQuery());
+                else
+                    qry = new SqlFieldsQuery(req.sqlQuery());
 
                 Iterator<Cache.Entry<String, String>> cur =
                     ctx.grid().cache(req.cacheName()).query(qry).iterator();
@@ -123,7 +130,7 @@ public class QueryCommandHandler extends GridRestCommandHandlerAdapter {
 
                 curs.put(qryId, cur);
 
-                List<Cache.Entry<String, String>> res = new ArrayList<>();
+                List res = new ArrayList<>();
 
                 CacheQueryResult response = new CacheQueryResult();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d4a29006/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 c65c49d..af86916 100644
--- a/modules/nodejs/src/main/js/apache-ignite.js
+++ b/modules/nodejs/src/main/js/apache-ignite.js
@@ -21,5 +21,6 @@ module.exports = {
     Server : require('./server.js').Server,
     Ignite : require('./ignite.js').Ignite,
     Compute : require('./compute.js').Compute,
-    SqlQuery : require('./sql-query.js').SqlQuery
+    SqlQuery : require('./sql-query.js').SqlQuery,
+    SqlFieldsQuery : require('./sql-fields-query.js').SqlFieldsQuery
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d4a29006/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 03f9231..991390a 100644
--- a/modules/nodejs/src/main/js/cache.js
+++ b/modules/nodejs/src/main/js/cache.js
@@ -16,6 +16,8 @@
  */
 
 var Server = require("./server").Server;
+var SqlFieldsQuery = require("./sql-fields-query").SqlFieldsQuery
+var SqlQuery = require("./sql-query").SqlQuery
 
 /**
  * Creates an instance of Cache
@@ -135,11 +137,16 @@ Cache.prototype.getAll = function(keys, callback) {
  */
 Cache.prototype.query = function(qry) {
     function onQueryExecute(qry, error, res) {
-        if (error) {
+        if (error !== null) {
             qry.error(error);
+            qry.end();
 
             return;
         }
+        console.log("Qry: " + qry.type());
+
+        console.log("Error: " + error);
+        console.log("Result: " + res);
 
         qry.page(res["items"]);
 
@@ -155,6 +162,25 @@ Cache.prototype.query = function(qry) {
         }
     }
 
+    if (qry.type() === "Sql") {
+        this._sqlQuery(qry, onQueryExecute);
+    }
+    else {
+        this._sqlFieldsQuery(qry, onQueryExecute);
+    }
+}
+
+Cache.prototype._sqlFieldsQuery = function(qry, onQueryExecute) {
+    var params = [Server.pair("cacheName", this._cacheName),
+        Server.pair("qry", qry.query()),
+        Server.pair("arg", qry.arguments()),
+        Server.pair("psz", qry.pageSize())];
+
+    this._server.runCommand("qryfieldsexecute", params,
+        onQueryExecute.bind(this, qry));
+}
+
+Cache.prototype._sqlQuery = function(qry, onQueryExecute) {
     var params = [Server.pair("cacheName", this._cacheName),
         Server.pair("qry", qry.query()),
         Server.pair("arg", qry.arguments()),

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d4a29006/modules/nodejs/src/main/js/server.js
----------------------------------------------------------------------
diff --git a/modules/nodejs/src/main/js/server.js b/modules/nodejs/src/main/js/server.js
index d0ce68f..488d497 100644
--- a/modules/nodejs/src/main/js/server.js
+++ b/modules/nodejs/src/main/js/server.js
@@ -83,10 +83,12 @@ Server.prototype.runCommand = function(cmdName, params, callback) {
         var fullResponseString = '';
 
         response.on('data', function (chunk) {
+            console.log("data:" + chunk);
             fullResponseString += chunk;
         });
 
         response.on('end', function () {
+            console.log("fullResponseString:" + fullResponseString);
             if (response.statusCode !== 200) {
                 if (response.statusCode === 401) {
                     callback.call(null, "Authentication failed. Status code 401.");
@@ -102,9 +104,11 @@ Server.prototype.runCommand = function(cmdName, params, callback) {
 
             try {
                 igniteResponse = JSON.parse(fullResponseString);
+                console.log("igniteResponse:" + igniteResponse);
             }
             catch (e) {
                 callback.call(null, e, null);
+
                 return;
             }
 
@@ -112,6 +116,7 @@ Server.prototype.runCommand = function(cmdName, params, callback) {
                 callback.call(null, igniteResponse.error, null)
             }
             else {
+                console.log("igniteResponse.response:" + igniteResponse.response);
                 callback.call(null, null, igniteResponse.response);
             }
         });

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d4a29006/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
new file mode 100644
index 0000000..02205cf
--- /dev/null
+++ b/modules/nodejs/src/main/js/sql-fields-query.js
@@ -0,0 +1,131 @@
+/*
+ * 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 {SqlFieldsQuery}
+ * @param {string} Sql query
+ */
+function SqlFieldsQuery(sql) {
+    this._qryType = "SqlFields";
+    this._sql = sql;
+    this._arg = [];
+    this._pageSz = 1;
+    this._type = null;
+    this._endFunc = function(res) {console.log("Empty end function is called [res=" + res + "]")};
+    this._pageFunc = function(res) {console.log("Empty page function is called [res=" + res + "]")}
+    this._errFunc = function(err) {console.log("Empty error function is called [err=" + err + "]")}
+}
+
+/**
+ * Set the callbacks for query events.
+ *
+ * @this {SqlFieldsQuery}
+ * @param {string} code Function code could be "end", "page", "error"
+ * @param function Functions "error" and "page" are one argument functions and "end" is function without arguments.
+ */
+SqlFieldsQuery.prototype.on = function(code, f) {
+    switch(code) {
+        case "end":
+            this._endFunc = f;
+
+            break;
+        case "page":
+            this._pageFunc = f;
+
+            break;
+        case "error" :
+            this._errFunc = f;
+
+            break;
+        default :
+            throw "Sql do not have method " + code;
+    }
+}
+
+/**
+ * @this {SqlFieldsQuery}
+ * @param res Query result
+ */
+SqlFieldsQuery.prototype.end = function(res) {
+    this._endFunc(res);
+}
+
+/**
+ * @this {SqlFieldsQuery}
+ * @param err Query error
+ */
+SqlFieldsQuery.prototype.error = function(err) {
+    this._errFunc(err);
+}
+
+/**
+ * @this {SqlFieldsQuery}
+ * @param res Query data
+ */
+SqlFieldsQuery.prototype.page = function(res) {
+    this._pageFunc(res);
+}
+
+/**
+ * @this {SqlFieldsQuery}
+ * @param {int} pageSz Page size.
+ */
+SqlFieldsQuery.prototype.setPageSize = function(pageSz) {
+    this._pageSize = pageSz;
+}
+
+/**
+ * @this {SqlFieldsQuery}
+ * @param args Arguments
+ */
+SqlFieldsQuery.prototype.setArguments = function(args) {
+    this._arg = args;
+}
+
+/**
+ * @this {SqlFieldsQuery}
+ * @returns Sql query
+ */
+SqlFieldsQuery.prototype.query = function() {
+    return this._sql;
+}
+
+/**
+ * @this {SqlFieldsQuery}
+ * @returns arguments
+ */
+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/d4a29006/modules/nodejs/src/main/js/sql-query.js
----------------------------------------------------------------------
diff --git a/modules/nodejs/src/main/js/sql-query.js b/modules/nodejs/src/main/js/sql-query.js
index 0c57638..5fae421 100644
--- a/modules/nodejs/src/main/js/sql-query.js
+++ b/modules/nodejs/src/main/js/sql-query.js
@@ -15,85 +15,22 @@
  * limitations under the License.
  */
 
+var SqlFieldsQuery = require("./sql-fields-query").SqlFieldsQuery
+
 /**
  * @this {SqlQuery}
  * @param {string} Sql query
  */
 function SqlQuery(sql) {
-    this._sql = sql;
-    this._arg = [];
-    this._pageSz = 1;
+    SqlFieldsQuery.apply(this, arguments);
     this._type = null;
-    this._endFunc = function(res) {console.log("Empty end function is called [res=" + res + "]")};
-    this._pageFunc = function(res) {console.log("Empty page function is called [res=" + res + "]")}
-    this._errFunc = function(err) {console.log("Empty error function is called [err=" + err + "]")}
-}
-
-/**
- * Set the callbacks for query events.
- *
- * @this {SqlQuery}
- * @param {string} code Function code could be "end", "page", "error"
- * @param function Functions "error" and "page" are one argument functions and "end" is function without arguments.
- */
-SqlQuery.prototype.on = function(code, f) {
-    switch(code) {
-        case "end":
-            this._endFunc = f;
-
-            break;
-        case "page":
-            this._pageFunc = f;
-
-            break;
-        case "error" :
-            this._errFunc = f;
-
-            break;
-        default :
-            throw "Sql do not have method " + code;
-    }
-}
-
-/**
- * @this {SqlQuery}
- * @param res Query result
- */
-SqlQuery.prototype.end = function(res) {
-    this._endFunc(res);
+    this._qryType = "Sql";
 }
 
-/**
- * @this {SqlQuery}
- * @param err Query error
- */
-SqlQuery.prototype.error = function(err) {
-    this._errFunc(err);
-}
-
-/**
- * @this {SqlQuery}
- * @param res Query data
- */
-SqlQuery.prototype.page = function(res) {
-    this._pageFunc(res);
-}
+SqlQuery.prototype = SqlFieldsQuery.prototype;
 
-/**
- * @this {SqlQuery}
- * @param {int} pageSz Page size.
- */
-SqlQuery.prototype.setPageSize = function(pageSz) {
-    this._pageSize = pageSz;
-}
+SqlQuery.prototype.constructor = SqlQuery;
 
-/**
- * @this {SqlQuery}
- * @param args Arguments
- */
-SqlQuery.prototype.setArguments = function(args) {
-    this._arg = args;
-}
 
 /**
  * @this {SqlQuery}
@@ -105,49 +42,10 @@ SqlQuery.prototype.setReturnType = function(type) {
 
 /**
  * @this {SqlQuery}
- * @returns Sql query
- */
-SqlQuery.prototype.query = function() {
-    return this._sql;
-}
-
-/**
- * @this {SqlQuery}
- * @returns arguments
- */
-SqlQuery.prototype.arguments = function() {
-    return this._arg;
-}
-
-/**
- * @this {SqlQuery}
- * @returns pageSize
- */
-SqlQuery.prototype.pageSize = function() {
-    return this._pageSz;
-}
-
-/**
- * @this {SqlQuery}
  * @returns Return class name
  */
 SqlQuery.prototype.returnType = function() {
     return this._type;
 }
 
-/**
- * @this {SqlQuery}
- * @returns "Sql"
- */
-SqlQuery.prototype.type = function() {
-    return SqlQuery.type();
-}
-
-/**
- * @returns "Sql"
- */
-SqlQuery.type = function() {
-    return "Sql"
-}
-
 exports.SqlQuery = SqlQuery;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d4a29006/modules/nodejs/src/test/java/org/apache/ignite/internal/NodeJsSqlQuery.java
----------------------------------------------------------------------
diff --git a/modules/nodejs/src/test/java/org/apache/ignite/internal/NodeJsSqlQuery.java b/modules/nodejs/src/test/java/org/apache/ignite/internal/NodeJsSqlQuery.java
index 54d6395..3fcf3ad 100644
--- a/modules/nodejs/src/test/java/org/apache/ignite/internal/NodeJsSqlQuery.java
+++ b/modules/nodejs/src/test/java/org/apache/ignite/internal/NodeJsSqlQuery.java
@@ -17,6 +17,14 @@
 
 package org.apache.ignite.internal;
 
+import org.apache.ignite.*;
+import org.apache.ignite.cache.affinity.*;
+import org.apache.ignite.cache.query.annotations.*;
+import org.apache.ignite.configuration.*;
+
+import java.io.*;
+import java.util.*;
+
 /**
  * Node js sql query test.
  */
@@ -44,4 +52,114 @@ public class NodeJsSqlQuery  extends NodeJsAbstractTest {
     public void testSqlQuery() throws Exception {
         runJsScript("testSqlQuery");
     }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSqlFieldsQuery() throws Exception {
+
+        CacheConfiguration<UUID, Organization> orgCacheCfg = new CacheConfiguration<>("organization");
+        orgCacheCfg.setIndexedTypes(UUID.class, Organization.class);
+
+        CacheConfiguration<AffinityKey<UUID>, Person> personCacheCfg = new CacheConfiguration<>("person");
+        personCacheCfg.setIndexedTypes(AffinityKey.class, Person.class);
+
+        IgniteCache<UUID, Organization> orgCache = grid(0).getOrCreateCache(orgCacheCfg);
+
+        Organization org1 = new Organization("ApacheIgnite");
+        Organization org2 = new Organization("Other");
+
+        orgCache.put(org1.id, org1);
+        orgCache.put(org2.id, org2);
+
+        IgniteCache<AffinityKey<UUID>, Person> personCache = grid(0).getOrCreateCache(personCacheCfg);
+
+        Person p1 = new Person(org1, "John", "Doe", 2000);
+        Person p2 = new Person(org1, "Jane", "Doe", 1000);
+        Person p3 = new Person(org2, "John", "Smith", 1000);
+        Person p4 = new Person(org2, "Jane", "Smith", 2000);
+
+        personCache.put(p1.key(), p1);
+        personCache.put(p2.key(), p2);
+        personCache.put(p3.key(), p3);
+        personCache.put(p4.key(), p4);
+
+        runJsScript("testSqlFieldsQuery");
+    }
+
+    /**
+     * Person class.
+     */
+    private static class Person implements Serializable {
+        /** Person ID (indexed). */
+        @QuerySqlField(index = true)
+        private UUID id;
+
+        /** Organization ID (indexed). */
+        @QuerySqlField(index = true)
+        private UUID orgId;
+
+        /** First name (not-indexed). */
+        @QuerySqlField
+        private String firstName;
+
+        /** Last name (not indexed). */
+        @QuerySqlField
+        private String lastName;
+
+        /** Salary (indexed). */
+        @QuerySqlField(index = true)
+        private double salary;
+
+        /** Custom cache key to guarantee that person is always collocated with its organization. */
+        private transient AffinityKey<UUID> key;
+
+        /**
+         * @param org Organization.
+         * @param firstName First name.
+         * @param lastName Last name.
+         * @param salary Salary.
+         */
+        Person(Organization org, String firstName, String lastName, double salary) {
+            id = UUID.randomUUID();
+
+            orgId = org.id;
+
+            this.firstName = firstName;
+            this.lastName = lastName;
+            this.salary = salary;
+        }
+
+        /**
+         * @return Custom affinity key to guarantee that person is always collocated with organization.
+         */
+        public AffinityKey<UUID> key() {
+            if (key == null)
+                key = new AffinityKey<>(id, orgId);
+
+            return key;
+        }
+    }
+
+    /**
+     * Organization class.
+     */
+    private static class Organization implements Serializable {
+        /** Organization ID (indexed). */
+        @QuerySqlField(index = true)
+        private UUID id;
+
+        /** Organization name (indexed). */
+        @QuerySqlField(index = true)
+        private String name;
+
+        /**
+         * @param name Organization name.
+         */
+        Organization(String name) {
+            id = UUID.randomUUID();
+
+            this.name = name;
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d4a29006/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 cb97619..4e61dc3 100644
--- a/modules/nodejs/src/test/js/test-query.js
+++ b/modules/nodejs/src/test/js/test-query.js
@@ -21,6 +21,7 @@ var assert = require("assert");
 
 var Ignite = require(TestUtils.scriptPath());
 var SqlQuery = Ignite.SqlQuery;
+var SqlFieldsQuery = Ignite.SqlFieldsQuery;
 
 testSqlQuery = function() {
     function sqlQuery(ignite, error) {
@@ -64,3 +65,41 @@ testSqlQuery = function() {
 
     TestUtils.startIgniteNode(put);
 }
+
+testSqlFieldsQuery = function() {
+    function sqlFieldsQuery(error, ignite) {
+        assert(error == null, "error on sqlfields query [err=" + error + "]");
+
+        var qry = new SqlFieldsQuery("select concat(firstName, ' ', lastName) from Person");
+
+        var fullRes = [];
+
+        qry.on("error", function(err) {
+                TestUtils.testFails();
+            });
+
+        qry.on("page", function(res) {
+            console.log("PAGE: " + res);
+            fullRes = fullRes.concat(res);
+        });
+
+        qry.on("end", function() {
+                console.log("END=" + fullRes);
+
+                assert(fullRes.length, 1, "Result length is not correct" +
+                    "[expected=1, val = " + fullRes.length + "]");
+
+                assert(fullRes[0]["key"] === "key0", "Result value for key is not correct "+
+                    "[expected=key0, real=" + fullRes[0]["key"] + "]");
+
+                assert(fullRes[0]["value"] === "val0", "Result value for key is not correct "+
+                    "[expected=val0, real=" + fullRes[0]["value"] + "]");
+
+                TestUtils.testDone();
+            });
+
+        ignite.cache("person").query(qry);
+    }
+
+    TestUtils.startIgniteNode(sqlFieldsQuery.bind(null));
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d4a29006/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 27db356..2269823 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
@@ -471,7 +471,8 @@ public class GridJettyRestHandler extends AbstractHandler {
                 break;
             }
 
-            case EXECUTE_SQL_QUERY: {
+            case EXECUTE_SQL_QUERY:
+            case EXECUTE_SQL_FIELDS_QUERY: {
                 RestSqlQueryRequest restReq0 = new RestSqlQueryRequest();
 
                 restReq0.sqlQuery((String)params.get("qry"));