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/01 12:13:50 UTC

[14/39] 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/3f86e4b4
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/3f86e4b4
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/3f86e4b4

Branch: refs/heads/ignite-964
Commit: 3f86e4b46dc445623b71bb90e37c9672ce41022f
Parents: 0643798
Author: ivasilinets <iv...@gridgain.com>
Authored: Tue Jun 30 15:30:56 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Tue Jun 30 15:30:56 2015 +0300

----------------------------------------------------------------------
 .../handlers/query/QueryCommandHandler.java     | 23 ++---
 modules/nodejs/src/main/js/cache.js             |  9 +-
 modules/nodejs/src/main/js/server.js            | 12 +--
 .../ignite/internal/NodeJsSqlQuerySelfTest.java | 96 ++++++++------------
 modules/nodejs/src/test/js/test-query.js        | 24 +++--
 .../http/jetty/GridJettyRestHandler.java        | 23 ++++-
 6 files changed, 97 insertions(+), 90 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3f86e4b4/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 a9f7b0f..a254115 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
@@ -45,7 +45,7 @@ public class QueryCommandHandler extends GridRestCommandHandlerAdapter {
     private static final AtomicLong qryIdGen = new AtomicLong();
 
     /** Current queries. */
-    private final ConcurrentHashMap<Long, Iterator<Cache.Entry<String, String>>> curs =
+    private final ConcurrentHashMap<Long, Iterator<Cache.Entry<Object, Object>>> curs =
         new ConcurrentHashMap<>();
 
     /**
@@ -100,14 +100,14 @@ public class QueryCommandHandler extends GridRestCommandHandlerAdapter {
         private RestSqlQueryRequest req;
 
         /** Queries cursors. */
-        private ConcurrentHashMap<Long, Iterator<Cache.Entry<String, String>>> curs;
+        private ConcurrentHashMap<Long, Iterator<Cache.Entry<Object, Object>>> curs;
 
         /**
          * @param ctx Kernal context.
          * @param req Execute query request.
          */
         public ExecuteQueryCallable(GridKernalContext ctx, RestSqlQueryRequest req,
-            ConcurrentHashMap<Long, Iterator<Cache.Entry<String, String>>> curs) {
+            ConcurrentHashMap<Long, Iterator<Cache.Entry<Object, Object>>> curs) {
             this.ctx = ctx;
             this.req = req;
             this.curs = curs;
@@ -129,7 +129,7 @@ public class QueryCommandHandler extends GridRestCommandHandlerAdapter {
                     ((SqlFieldsQuery)qry).setArgs(req.arguments());
                 }
 
-                Iterator<Cache.Entry<String, String>> cur =
+                Iterator<Cache.Entry<Object, Object>> cur =
                     ctx.grid().cache(req.cacheName()).query(qry).iterator();
 
                 long qryId = qryIdGen.getAndIncrement();
@@ -140,8 +140,9 @@ public class QueryCommandHandler extends GridRestCommandHandlerAdapter {
 
                 CacheQueryResult response = new CacheQueryResult();
 
-                for (int i = 0; i < req.pageSize() && cur.hasNext(); ++i)
+                for (int i = 0; i < req.pageSize() && cur.hasNext(); ++i) {
                     res.add(cur.next());
+                }
 
                 response.setItems(res);
 
@@ -174,14 +175,14 @@ public class QueryCommandHandler extends GridRestCommandHandlerAdapter {
         private RestSqlQueryRequest req;
 
         /** Queries cursors. */
-        private ConcurrentHashMap<Long, Iterator<Cache.Entry<String, String>>> curs;
+        private ConcurrentHashMap<Long, Iterator<Cache.Entry<Object, Object>>> curs;
 
         /**
          * @param ctx Kernal context.
          * @param req Execute query request.
          */
         public FetchQueryCallable(GridKernalContext ctx, RestSqlQueryRequest req,
-            ConcurrentHashMap<Long, Iterator<Cache.Entry<String, String>>> curs) {
+            ConcurrentHashMap<Long, Iterator<Cache.Entry<Object, Object>>> curs) {
             this.ctx = ctx;
             this.req = req;
             this.curs = curs;
@@ -190,19 +191,19 @@ public class QueryCommandHandler extends GridRestCommandHandlerAdapter {
         /** {@inheritDoc} */
         @Override public GridRestResponse call() throws Exception {
             try {
-                Iterator<Cache.Entry<String, String>> cur = curs.get(req.queryId());
+                Iterator<Cache.Entry<Object, Object>> cur = curs.get(req.queryId());
 
                 if (cur == null)
                     return new GridRestResponse(GridRestResponse.STATUS_FAILED,
                         "Cannot find query [qryId=" + req.queryId() + "]");
 
-                List<Cache.Entry<String, String>> res = new ArrayList<>();
+                List res = new ArrayList<>();
 
                 CacheQueryResult response = new CacheQueryResult();
 
-                for (int i = 0; i < req.pageSize() && cur.hasNext(); ++i)
+                for (int i = 0; i < req.pageSize() && cur.hasNext(); ++i) {
                     res.add(cur.next());
-
+                }
                 response.setItems(res);
 
                 response.setLast(!cur.hasNext());

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3f86e4b4/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 0e08c82..8ff6ee3 100644
--- a/modules/nodejs/src/main/js/cache.js
+++ b/modules/nodejs/src/main/js/cache.js
@@ -129,7 +129,9 @@ Cache.prototype.query = function(qry) {
         }
         else {
             var command = this._createCommand("qryfetch");
+
             command.addParam("qryId", res.queryId).addParam("psz", qry.pageSize());
+
             this._server.runCommand(command, onQueryExecute.bind(this, qry));
         }
     }
@@ -144,7 +146,8 @@ Cache.prototype.query = function(qry) {
 
 Cache.prototype._sqlFieldsQuery = function(qry, onQueryExecute) {
     var command = this._createQueryCommand("qryfieldsexecute", qry);
-    command.addParams("arg", qry.arguments());
+
+    command.setPostData(JSON.stringify({"arg" : qry.arguments()}));
 
     this._server.runCommand(command, onQueryExecute.bind(this, qry));
 }
@@ -158,9 +161,11 @@ Cache.prototype._sqlQuery = function(qry, onQueryExecute) {
     }
 
     var command = this._createQueryCommand("qryexecute", qry);
-    command.addParams("arg", qry.arguments());
+
     command.addParam("type", qry.returnType());
 
+    command.setPostData(JSON.stringify({"arg" : qry.arguments()}));
+
     this._server.runCommand(command, onQueryExecute.bind(this, qry));
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3f86e4b4/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 3cb98c1..e49ed83 100644
--- a/modules/nodejs/src/main/js/server.js
+++ b/modules/nodejs/src/main/js/server.js
@@ -67,22 +67,17 @@ Server.prototype.runCommand = function(cmd, callback) {
 
     var http = require('http');
 
-    var commHeaders = this._signature();
-
-    if (cmd._isPost()) {
-        commHeaders["JSONObject"] = "true";
-    }
-
     var options = {
         host: this._host,
         port: this._port,
         method : cmd._method(),
         path: "/ignite?" + requestQry,
-        headers: commHeaders
+        headers: this._signature()
     };
 
     if (cmd._isPost()) {
         options.headers['Content-Length'] = cmd.postData().length;
+        options.headers['JSONObject'] = "true";
     }
 
     function streamCallback(response) {
@@ -134,6 +129,7 @@ Server.prototype.runCommand = function(cmd, callback) {
     if (cmd._isPost()) {
         request.write(cmd.postData());
     }
+
     request.end();
 }
 
@@ -155,7 +151,7 @@ Server.prototype.checkConnection = function(callback) {
  */
 Server.prototype._signature = function() {
     if (!this._secretKey) {
-        return {};
+        return "";
     }
 
     var loadTimeInMS = Date.now();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3f86e4b4/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 97d9852..8d33668 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
@@ -75,45 +75,30 @@ public class NodeJsSqlQuerySelfTest extends NodeJsAbstractTest {
      * Init cache.
      */
     private void initCache() {
-        CacheConfiguration<UUID, Organization> orgCacheCfg = new CacheConfiguration<>("organization");
-        orgCacheCfg.setIndexedTypes(UUID.class, Organization.class);
+        CacheConfiguration<UUID, Person> personCacheCfg = new CacheConfiguration<>("person");
+        personCacheCfg.setIndexedTypes(UUID.class, Person.class);
 
-        CacheConfiguration<AffinityKey<UUID>, Person> personCacheCfg = new CacheConfiguration<>("person");
-        personCacheCfg.setIndexedTypes(AffinityKey.class, Person.class);
+        IgniteCache<UUID, Person> personCache = grid(0).getOrCreateCache(personCacheCfg);
 
-        IgniteCache<UUID, Organization> orgCache = grid(0).getOrCreateCache(orgCacheCfg);
+        Person p1 = new Person("John", "Doe", 2000);
+        Person p2 = new Person("Jane", "Doe", 1000);
+        Person p3 = new Person("John", "Smith", 1000);
+        Person p4 = new Person("Jane", "Smith", 2000);
 
-        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);
+        personCache.put(p4.getId(), p1);
+        personCache.put(p4.getId(), p2);
+        personCache.put(p4.getId(), p3);
+        personCache.put(p4.getId(), p4);
     }
 
     /**
      * Person class.
      */
-    private static class Person implements Serializable {
+    public 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;
@@ -126,55 +111,50 @@ public class NodeJsSqlQuerySelfTest extends NodeJsAbstractTest {
         @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) {
+        Person( 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);
+        public void setFirstName(String firstName) {
+            this.firstName = firstName;
+        }
 
-            return key;
+        public String getFirstName() {
+            return firstName;
         }
-    }
 
-    /**
-     * Organization class.
-     */
-    private static class Organization implements Serializable {
-        /** Organization ID (indexed). */
-        @QuerySqlField(index = true)
-        private UUID id;
+        public void setLastName(String lastName) {
+            this.lastName = lastName;
+        }
 
-        /** Organization name (indexed). */
-        @QuerySqlField(index = true)
-        private String name;
+        public String getLastName() {
+            return lastName;
+        }
 
-        /**
-         * @param name Organization name.
-         */
-        Organization(String name) {
-            id = UUID.randomUUID();
+        public void setId(UUID id) {
+            this.id = id;
+        }
+
+        public void setSalary(double salary) {
+            this.salary = salary;
+        }
+
+        public double getSalary() {
+
+            return salary;
+        }
 
-            this.name = name;
+        public UUID getId() {
+            return id;
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3f86e4b4/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 77bbad1..c407816 100644
--- a/modules/nodejs/src/test/js/test-query.js
+++ b/modules/nodejs/src/test/js/test-query.js
@@ -110,24 +110,28 @@ testSqlQueryWithParams = function() {
         var fullRes = [];
 
         qry.on("page", function(res) {
+            console.log("!!!!!Page:" + res);
+            console.log("!!!!!Page2:" + res);
+
             fullRes = fullRes.concat(res);
         });
 
         qry.on("end", function(err) {
-                assert(err === null, "Error on query [err=" + err + "].");
+            console.log("RES:" + fullRes);
 
-                //TODO:
-                assert(fullRes.length, 1, "Result length is not correct" +
-                    "[expected=1, val = " + fullRes.length + "]");
+            assert(err === null, "Error on query [err=" + err + "].");
 
-                assert(fullRes[0].indexOf("Jane Doe") > -1,
-                    "Result does not contain Jane Doe [res=" + fullRes[0] + "]");
+            //TODO:
+            assert(fullRes.length, 2, "Result length is not correct" +
+                "[expected=1, val = " + fullRes.length + "]");
+
+            assert(fullRes[0].indexOf("Jane Doe") > -1,
+                "Result does not contain Jane Doe [res=" + fullRes[0] + "]");
 
-                TestUtils.testDone();
-            });
+            TestUtils.testDone();
+        });
 
         ignite.cache("person").query(qry);
     }
 
-    TestUtils.startIgniteNode(sqlFieldsQuery.bind(null));
-}
+    TestUtils.startIgniteN
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3f86e4b4/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 0af9bf7..e37f422 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
@@ -506,9 +506,30 @@ public class GridJettyRestHandler extends AbstractHandler {
                 RestSqlQueryRequest restReq0 = new RestSqlQueryRequest();
 
                 restReq0.sqlQuery((String)params.get("qry"));
-                List<Object> args = values("arg", params);
+
+                StringBuilder builder = new StringBuilder();
+
+                Scanner reader = null;
+
+                try {
+                    reader = new Scanner(req.getReader());
+                }
+                catch (IOException e) {
+                    throw new IgniteCheckedException(e);
+                }
+
+                while (reader.hasNext())
+                    builder.append(reader.next() + "\n");
+
+                JSONObject o = JSONObject.fromObject(builder.toString());
+                System.out.println("ARGUMENTS " + o.get("arg"));
+
+                List<Object> args = (List<Object>)o.get("arg");
+
                 restReq0.arguments(args.toArray());
+
                 restReq0.typeName((String)params.get("type"));
+
                 restReq0.pageSize(Integer.parseInt((String) params.get("psz")));
                 restReq0.cacheName((String)params.get("cacheName"));