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/23 19:47:27 UTC

[01/20] incubator-ignite git commit: #ignite-965: map and reduce are executed on server.

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-965 01c9c3acb -> 47276a81e


#ignite-965: map and reduce are executed on server.


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

Branch: refs/heads/ignite-965
Commit: d481ea8c76f9de8e0e94e6a2ee7a46d3788c1442
Parents: 01c9c3a
Author: ivasilinets <iv...@gridgain.com>
Authored: Tue Jun 23 18:08:27 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Tue Jun 23 18:08:27 2015 +0300

----------------------------------------------------------------------
 .../IgniteComputeTaskCommandHandler.java        | 102 ++++++++++++++-----
 .../rest/request/RestComputeTaskRequest.java    |  38 +++++--
 .../scripting/IgniteScriptProcessor.java        |  22 +++-
 modules/nodejs/src/main/js/apache-ignite.js     |   3 +-
 modules/nodejs/src/main/js/compute-task.js      |  38 -------
 modules/nodejs/src/main/js/compute.js           |  75 ++------------
 modules/nodejs/src/test/js/test-compute.js      |  37 ++++---
 modules/nodejs/src/test/js/test-utils.js        |  10 +-
 .../http/jetty/GridJettyRestHandler.java        |  17 +---
 9 files changed, 161 insertions(+), 181 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d481ea8c/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/compute/IgniteComputeTaskCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/compute/IgniteComputeTaskCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/compute/IgniteComputeTaskCommandHandler.java
index a3708ab..7329a5b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/compute/IgniteComputeTaskCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/compute/IgniteComputeTaskCommandHandler.java
@@ -25,12 +25,11 @@ import org.apache.ignite.internal.processors.rest.*;
 import org.apache.ignite.internal.processors.rest.handlers.*;
 import org.apache.ignite.internal.processors.rest.request.*;
 import org.apache.ignite.internal.util.future.*;
-import org.apache.ignite.internal.util.typedef.T3;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.resources.*;
 import org.jetbrains.annotations.*;
 
-import javax.script.ScriptException;
+import javax.script.*;
 import java.util.*;
 
 import static org.apache.ignite.internal.processors.rest.GridRestCommand.*;
@@ -64,25 +63,28 @@ public class IgniteComputeTaskCommandHandler extends GridRestCommandHandlerAdapt
 
         final RestComputeTaskRequest req0 = (RestComputeTaskRequest) req;
 
-        List<T3<String, String, String>> mapping =  req0.mapping();
-
-        Object res = ctx.grid().compute().execute(new JsTask(mapping, ctx), null);
+        Object res = ctx.grid().compute().execute(new JsTask(req0.mapFunc(), req0.argument(), req0.reduceFunc(), ctx), null);
 
         return new GridFinishedFuture<>(new GridRestResponse(res));
     }
 
     private static class JsTask extends ComputeTaskAdapter<String, Object> {
-        /** Mapping. */
-        private List<T3<String, String, String>> mapping;
+        /** Mapping function. */
+        private String mapFunc;
+
+        private String reduceFunc;
 
         /** Grid kernal context. */
         private GridKernalContext ctx;
 
+        private String arg;
+
         /**
-         * @param mapping Task mapping.
          */
-        public JsTask(List<T3<String, String, String>> mapping, GridKernalContext ctx) {
-            this.mapping = mapping;
+        public JsTask(String mapFunc, String arg, String reduceFunc, GridKernalContext ctx) {
+            this.mapFunc = mapFunc;
+            this.reduceFunc = reduceFunc;
+            this.arg = arg;
             this.ctx = ctx;
         }
 
@@ -90,27 +92,68 @@ public class IgniteComputeTaskCommandHandler extends GridRestCommandHandlerAdapt
         @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> nodes, String arg) {
             Map<ComputeJob, ClusterNode> map = new HashMap<>();
 
-            for (final T3<String, String, String> job : mapping) {
-                UUID nodeId = UUID.fromString(job.get2());
+            String nodesIds = "[";
 
-                ClusterNode node = ctx.grid().cluster().node(nodeId);
+            for (ClusterNode node : nodes)
+                nodesIds += "\""  + node.id().toString() + "\"" + ",";
 
-                map.put(new ComputeJobAdapter() {
-                    /** Ignite. */
-                    @IgniteInstanceResource
-                    private Ignite ignite;
+            nodesIds = nodesIds.substring(0, nodesIds.length() - 1) + "]";
 
-                    @Override public Object execute() throws IgniteException {
-                        System.out.println("Compute job on node " + ignite.cluster().localNode().id());
+            try {
+                String newMap = new String("function () {\n" +
+                    "   var res = [];\n" +
+                    "   var resCont = function(f, args, nodeId) {\n" +
+                    "       res.push([f.toString(), args, nodeId])\n" +
+                    "   }\n" +
+                    "   var locF = " + mapFunc + "; \n locF(" +
+                        nodesIds + ", " +
+                    "\"" + this.arg + "\"" +
+                    ", resCont.bind(null)" + ");\n" +
+                    "   return res;\n" +
+                    "}");
 
-                        try {
-                            return ((IgniteKernal)ignite).context().scripting().runJS(job.get1(), job.get3());
-                        }
-                        catch (ScriptException e) {
-                            throw new IgniteException(e);
+                List mapRes = (List)ctx.scripting().runJS(newMap);
+
+                for (Object arr : mapRes) {
+                    Object[] nodeTask = ((List)arr).toArray();
+
+                    final String func = (String)nodeTask[0];
+
+                    final List argv = (List) nodeTask[1];
+
+                    String nodeIdStr = (String) nodeTask[2];
+
+                    UUID nodeId = UUID.fromString(nodeIdStr);
+
+                    ClusterNode node = ctx.grid().cluster().node(nodeId);
+
+                    map.put(new ComputeJobAdapter() {
+                        /** Ignite. */
+                        @IgniteInstanceResource
+                        private Ignite ignite;
+
+                        @Override public Object execute() throws IgniteException {
+                            System.out.println("Compute job on node " + ignite.cluster().localNode().id());
+                            try {
+                                String[] argv1 = new String[argv.size()];
+
+                                for (int i = 0; i < argv1.length; ++i)
+                                    argv1[i] = "\"" + argv.get(i).toString() + "\"";
+
+                                return ctx.scripting().runJS(func, argv1);
+                            }
+                            catch (Exception e) {
+                                throw new IgniteException(e);
+                            }
                         }
-                    }
-                }, node);
+                    }, node);
+
+                }
+            }
+            catch (ScriptException e) {
+                throw new IgniteException(e);
+            }
+            finally {
             }
 
             return map;
@@ -123,7 +166,12 @@ public class IgniteComputeTaskCommandHandler extends GridRestCommandHandlerAdapt
             for (ComputeJobResult res : results)
                 data.add(res.getData());
 
-            return data;
+            try {
+                return ctx.scripting().runJS(reduceFunc, new String[] {data.toString()});
+            }
+            catch (ScriptException e) {
+                throw new IgniteException(e);
+            }
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d481ea8c/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/RestComputeTaskRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/RestComputeTaskRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/RestComputeTaskRequest.java
index bae7ce5..89e04da 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/RestComputeTaskRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/RestComputeTaskRequest.java
@@ -26,19 +26,35 @@ import java.util.*;
  */
 public class RestComputeTaskRequest extends GridRestRequest {
     /** Mapping tasks to nodes. */
-    private List<T3<String, String, String>> mapping;
+    private String mapFunc;
 
-    /**
-     * @param mapping Mapping tasks to nodes.
-     */
-    public void mapping(List<T3<String, String, String>> mapping) {
-        this.mapping = mapping;
+    /** Function argument. */
+    private String arg;
+
+    private String reduceFunc;
+
+    public void reduceFunc(String reduceFunc) {
+        this.reduceFunc = reduceFunc;
+    }
+
+    public String reduceFunc() {
+
+        return reduceFunc;
+    }
+
+    public String mapFunc() {
+        return mapFunc;
+    }
+
+    public String argument() {
+        return arg;
+    }
+
+    public void mapFunc(String mapFunc) {
+        this.mapFunc = mapFunc;
     }
 
-    /**
-     * @return Mapping tasks to nodes.
-     */
-    public List<T3<String, String, String>> mapping() {
-        return mapping;
+    public void argument(String arg) {
+        this.arg = arg;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d481ea8c/modules/core/src/main/java/org/apache/ignite/internal/processors/scripting/IgniteScriptProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/scripting/IgniteScriptProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/scripting/IgniteScriptProcessor.java
index 54004de..99c206e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/scripting/IgniteScriptProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/scripting/IgniteScriptProcessor.java
@@ -20,8 +20,10 @@ package org.apache.ignite.internal.processors.scripting;
 import org.apache.ignite.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.*;
+import org.apache.ignite.internal.util.typedef.*;
 
 import javax.script.*;
+import java.util.*;
 
 /**
  * Ignite scripting manager.
@@ -46,7 +48,7 @@ public class IgniteScriptProcessor extends GridProcessorAdapter {
      * @throws ScriptException If script failed.
      */
     public Object run(String engName, String script) throws ScriptException {
-        if (engName.equals(JAVA_SCRIPT_ENGINE_NAME))
+        if (!engName.equals(JAVA_SCRIPT_ENGINE_NAME))
             throw new IgniteException("Engine is not supported. [engName=" + engName + "]");
 
         return runJS(script);
@@ -54,9 +56,11 @@ public class IgniteScriptProcessor extends GridProcessorAdapter {
 
     /**
      * @param script Script.
+     * @param args Arguments.
+     * @return Script result.
      * @throws ScriptException If script failed.
      */
-    public Object runJS(String script, String args) throws ScriptException {
+    public Object runJS(String script, String[] args) throws ScriptException {
         ScriptEngine engine = factory.getEngineByName("JavaScript");
 
         Bindings b = engine.createBindings();
@@ -65,13 +69,23 @@ public class IgniteScriptProcessor extends GridProcessorAdapter {
 
         engine.setBindings(b, ScriptContext.ENGINE_SCOPE);
 
-        script = "(" + script + ")(" + args + ");";
+        script = "(" + script + ")(" ;
+
+        for (int i = 0; i < args.length; ++i)
+            script += args[i] + (i < args.length - 1 ? "," : "");
+
+        script += ");";
 
         return engine.eval(script);
     }
 
+    /**
+     * @param script Script.
+     * @return Script result.
+     * @throws ScriptException If script failed.
+     */
     public Object runJS(String script) throws ScriptException {
-        return runJS(script, "");
+        return runJS(script, new String[]{""});
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d481ea8c/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 f90cf68..0df6160 100644
--- a/modules/nodejs/src/main/js/apache-ignite.js
+++ b/modules/nodejs/src/main/js/apache-ignite.js
@@ -20,6 +20,5 @@ module.exports = {
   Ignition : require('./ignition.js').Ignition,
   Server : require('./server.js').Server,
   Ignite : require('./ignite.js').Ignite,
-  Compute : require('./compute.js').Compute,
-  ComputeJob: require('./compute.js').ComputeJob
+  Compute : require('./compute.js').Compute
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d481ea8c/modules/nodejs/src/main/js/compute-task.js
----------------------------------------------------------------------
diff --git a/modules/nodejs/src/main/js/compute-task.js b/modules/nodejs/src/main/js/compute-task.js
deleted file mode 100644
index d13f361..0000000
--- a/modules/nodejs/src/main/js/compute-task.js
+++ /dev/null
@@ -1,38 +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.
- */
-
-/**
- * @constructor
- * @this {ComputeTask}
- */
-function ComputeTask() {
-}
-
-/**
- * @param {string[]} nodes Nodes id
- * @param {string} arg Argument
- * @returns {Object.<string, Cache~onGet>} Map of grid jobs assigned to subgrid node. Unless {@link ComputeTaskContinuousMapper} is
- * injected into task, if {@code null} or empty map is returned, exception will be thrown
- */
-ComputeTask.prototype.map = function(nodes, arg) {
-}
-
-/**
- * @param {string[]} results Results
- */
-ComputeTask.prototype.reduce = function(results) {
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d481ea8c/modules/nodejs/src/main/js/compute.js
----------------------------------------------------------------------
diff --git a/modules/nodejs/src/main/js/compute.js b/modules/nodejs/src/main/js/compute.js
index fb38e7a..7a5f808 100644
--- a/modules/nodejs/src/main/js/compute.js
+++ b/modules/nodejs/src/main/js/compute.js
@@ -16,7 +16,6 @@
  */
 
 var Server = require("./server").Server;
-var ComputeTask = require("./compute-task").ComputeTask;
 
 /**
  * @constructor
@@ -72,76 +71,14 @@ Compute.prototype._escape = function(f) {
  * @param {string} arg  Argument
  * @param {} callback Callback
  */
-Compute.prototype.execute = function(task, arg, callback) {
-  this._nodes(this._onNodesExecute.bind(this, task, arg, callback));
-}
-
-Compute.prototype._nodes = function(callback) {
-  this._server.runCommand("top", [Server.pair("mtr", "false"), Server.pair("attr", "false")],
-    this._onNodes.bind(this, callback))
-}
-
-Compute.prototype._onNodes = function(callback, error, results) {
-  if (error) {
-    callback.call(null, error, null);
-
-    return;
-  }
-
-  var nodes = [];
-
-  for (var res of results) {
-    nodes.push(res["nodeId"])
-  }
-
-  callback.call(null, null, nodes);
-}
-
-Compute.prototype._onNodesExecute = function(task, arg, callback, err, nodes) {
-  if (err) {
-      callback.call(null, error, null);
-
-      return;
-  }
-
-  var computeJobList = task.map(nodes, arg);
+Compute.prototype.execute = function(map, reduce, arg, callback) {
+   var params = [];
 
-  var params = [];
-  var i = 1;
+    params.push(Server.pair("map", this._escape(map)));
+    params.push(Server.pair("reduce", this._escape(reduce)));
+    params.push(Server.pair("arg", this._escape(arg)));
 
-  console.log("TASK" + computeJobList);
-  for (var job of computeJobList) {
-    params.push(Server.pair("f" + i, this._escape(job.func)));
-    params.push(Server.pair("args" + i,  JSON.stringify(job.args)));
-    params.push(Server.pair("n" + i, job.node));
-    i++;
-  }
-
-  this._server.runCommand("exectask", params, this._onResExecute.bind(this, task, callback));
-}
-
-
-Compute.prototype._onResExecute = function(task, callback, err, results) {
-  if (err) {
-    callback.call(null, err, null);
-
-    return;
-  }
-
-  console.log("ON RES EXEC = " + results);
-
-  var res = task.reduce(results);
-
-  callback.call(null, null, res);
+    this._server.runCommand("exectask", params, callback);
 }
 
 exports.Compute = Compute
-
-
-function ComputeJob(func, args, node) {
-    this.func = func;
-    this.args = args;
-    this.node = node;
-}
-
-exports.ComputeJob = ComputeJob;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d481ea8c/modules/nodejs/src/test/js/test-compute.js
----------------------------------------------------------------------
diff --git a/modules/nodejs/src/test/js/test-compute.js b/modules/nodejs/src/test/js/test-compute.js
index 87a284c..c444380 100644
--- a/modules/nodejs/src/test/js/test-compute.js
+++ b/modules/nodejs/src/test/js/test-compute.js
@@ -17,10 +17,6 @@
 
 var TestUtils = require("./test-utils").TestUtils;
 
-var Apache = require(TestUtils.scriptPath());
-var Cache = Apache.Cache;
-var Server = Apache.Server;
-
 var assert = require("assert");
 
 testComputeAffinityRun = function() {
@@ -32,11 +28,7 @@ testComputeAffinityCall = function() {
 }
 
 testComputeExecute = function() {
-  var CharacterCountTask = require("./simple-compute-task").CharacterCountTask
-
-  var task = new CharacterCountTask();
-
-  TestUtils.startIgniteNode(onStart1.bind(null, task));
+  TestUtils.startIgniteNode(onStart1);
 }
 
 function onStart(locOnPut, error, ignite) {
@@ -92,10 +84,31 @@ function onError1(error, res) {
   TestUtils.testDone();
 }
 
-function onStart1(task, error, ignite) {
-  var comp = ignite.compute();
+function onStart1(error, ignite) {
+  var map = function(nodes, arg, emit) {
+    var words = arg.split(" ");
+
+    for (var i = 0; i < words.length; i++) {
+      var f = function (word) {
+        println(">>> Printing " + word);
+
+        return word.length;
+      };
+
+      emit(f, [words[i]], nodes[i %  nodes.length]);
+    }
+  };
+
+  var reduce = function(results) {
+    var sum = 0;
+
+    for (var i = 0; i < results.length; ++i)
+     sum += parseInt(results[i], 10);
+
+    return sum;
+  };
 
-  comp.execute(task, "Hi Alice", onComputeResult);
+  ignite.compute().execute(map, reduce, "Hi Alice", onComputeResult);
 }
 
 function onComputeResult(error, res) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d481ea8c/modules/nodejs/src/test/js/test-utils.js
----------------------------------------------------------------------
diff --git a/modules/nodejs/src/test/js/test-utils.js b/modules/nodejs/src/test/js/test-utils.js
index 135da8c..ee83d7f 100644
--- a/modules/nodejs/src/test/js/test-utils.js
+++ b/modules/nodejs/src/test/js/test-utils.js
@@ -129,8 +129,9 @@ TestUtils.testDone = function() {
  * @param {Ignition~onStart} callback Called on connect
  */
 TestUtils.startIgniteNode = function(callback) {
-  var Apache = require(TestUtils.scriptPath());
-  var Ignition = Apache.Ignition;
+  var Ignite = require(TestUtils.scriptPath());
+  var Ignition = Ignite.Ignition;
+
   Ignition.start(['127.0.0.1:9095'], null, callback);
 }
 
@@ -141,8 +142,9 @@ TestUtils.startIgniteNode = function(callback) {
  * @param {Ignition~onStart} callback Called on connect
  */
 TestUtils.startIgniteNodeWithKey = function(secretKey, callback) {
-  var Apache = require(TestUtils.scriptPath());
-  var Ignition = Apache.Ignition;
+  var Ignite = require(TestUtils.scriptPath());
+  var Ignition = Ignite.Ignition;
+
   Ignition.start(['127.0.0.1:9095'], secretKey, callback);
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d481ea8c/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 5fe4cd9..7d71530 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
@@ -462,20 +462,9 @@ public class GridJettyRestHandler extends AbstractHandler {
             case EXECUTE_TASK: {
                 RestComputeTaskRequest restReq0 = new RestComputeTaskRequest();
 
-                List<Object> funcs = values("f", params);
-                List<Object> nodes = values("n", params);
-
-                List<Object> args = values("args", params);
-
-                assert funcs.size() == nodes.size();
-
-                List<T3<String, String, String>> mapping = new ArrayList<>();
-
-
-                for (int i = 0; i < funcs.size(); ++i)
-                    mapping.add(new T3((String) funcs.get(i), (String)nodes.get(i), (String)args.get(i)));
-
-                restReq0.mapping(mapping);
+                restReq0.mapFunc((String)params.get("map"));
+                restReq0.argument((String) params.get("arg"));
+                restReq0.reduceFunc((String) params.get("reduce"));
 
                 restReq = restReq0;
 


[06/20] incubator-ignite git commit: #ignite-965: codestyle IgniteComputeTaskCommandHandler.

Posted by sb...@apache.org.
#ignite-965: codestyle IgniteComputeTaskCommandHandler.


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

Branch: refs/heads/ignite-965
Commit: 5e6056256924601a759f2f7b3017bf3a8d1c7cd5
Parents: 57e9845
Author: ivasilinets <iv...@gridgain.com>
Authored: Tue Jun 23 19:40:47 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Tue Jun 23 19:40:47 2015 +0300

----------------------------------------------------------------------
 .../rest/handlers/compute/IgniteComputeTaskCommandHandler.java   | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5e605625/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/compute/IgniteComputeTaskCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/compute/IgniteComputeTaskCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/compute/IgniteComputeTaskCommandHandler.java
index 33cd45b..59feae7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/compute/IgniteComputeTaskCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/compute/IgniteComputeTaskCommandHandler.java
@@ -101,9 +101,9 @@ public class IgniteComputeTaskCommandHandler extends GridRestCommandHandlerAdapt
         @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> nodes, String arg) {
             Map<ComputeJob, ClusterNode> map = new HashMap<>();
 
-            List jsMapResult = (List)ctx.scripting().runJSFunction(wrapMapperFunction(nodes));
+            List jsMapRes = (List)ctx.scripting().runJSFunction(wrapMapperFunction(nodes));
 
-            for (Object jobMapping : jsMapResult) {
+            for (Object jobMapping : jsMapRes) {
                 List task = (List)jobMapping;
 
                 final String func = (String)task.get(0);


[19/20] incubator-ignite git commit: #ignite-965: code style test-ignition.js

Posted by sb...@apache.org.
#ignite-965: code style test-ignition.js


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

Branch: refs/heads/ignite-965
Commit: d7e546502d90922cfe4efb20c236505b4e718106
Parents: e7375e9
Author: ivasilinets <iv...@gridgain.com>
Authored: Tue Jun 23 20:33:37 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Tue Jun 23 20:33:37 2015 +0300

----------------------------------------------------------------------
 modules/nodejs/src/test/js/test-ignition.js | 91 +++++++++++-----------
 modules/nodejs/src/test/js/test-key.js      |  4 -
 modules/nodejs/src/test/js/test-node.xml    | 97 ------------------------
 modules/nodejs/src/test/js/test-runner.js   |  1 +
 modules/nodejs/src/test/js/test-utils.js    |  6 +-
 5 files changed, 52 insertions(+), 147 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d7e54650/modules/nodejs/src/test/js/test-ignition.js
----------------------------------------------------------------------
diff --git a/modules/nodejs/src/test/js/test-ignition.js b/modules/nodejs/src/test/js/test-ignition.js
index c43e856..3823109 100644
--- a/modules/nodejs/src/test/js/test-ignition.js
+++ b/modules/nodejs/src/test/js/test-ignition.js
@@ -16,76 +16,77 @@
  */
 
 var TestUtils = require("./test-utils").TestUtils;
-var Apache = require(TestUtils.scriptPath());
-var Ignition = Apache.Ignition;
 
-testIgnitionFail = function ()  {
-    Ignition.start(['127.0.0.3:9091', '127.0.0.1:9092'], null, onConnect);
+var Ignite = require(TestUtils.scriptPath());
+var Ignition = Ignite.Ignition;
 
-    function onConnect(error, server) {
-        if (error) {
-            if (error.indexOf("Cannot connect to servers.") == -1)
-                TestUtils.testFails("Incorrect error message: " + error);
-            else
-                TestUtils.testDone();
+var assert = require("assert");
 
-            return;
-        }
+testIgnitionFail = function ()  {
+  Ignition.start(['127.0.0.3:9091', '127.0.0.1:9092'], null, onConnect);
+
+  function onConnect(error, server) {
+    if (error) {
+      if (error.indexOf("Cannot connect to servers.") == -1) {
+        TestUtils.testFails("Incorrect error message: " + error);
+      }
+      else {
+        TestUtils.testDone();
+      }
 
-        TestUtils.testFails("Test should fail.");
+      return;
     }
+
+    TestUtils.testFails("Test should fail.");
+  }
 }
 
 ignitionStartSuccess = function() {
-    Ignition.start(['127.0.0.0:9095', '127.0.0.1:9095'], null, onConnect);
+  Ignition.start(['127.0.0.0:9095', '127.0.0.1:9095'], null, onConnect);
 
-    function onConnect(error, server) {
-        if (error) {
-            TestUtils.testFails(error);
+  function onConnect(error, server) {
+    if (error) {
+      TestUtils.testFails(error);
 
-            return;
-        }
-
-        TestUtils.testDone();
+      return;
     }
+
+    TestUtils.testDone();
+  }
 }
 
 ignitionStartSuccessWithSeveralPorts = function() {
-    Ignition.start(['127.0.0.1:9090..9100'], null, onConnect);
+  Ignition.start(['127.0.0.1:9090..9100'], null, onConnect);
 
-    function onConnect(error, ignite) {
-        if (error) {
-            TestUtils.testFails(error);
+  function onConnect(error, ignite) {
+    if (error) {
+      TestUtils.testFails(error);
 
-            return;
-        }
-
-        var assert = require("assert");
+      return;
+    }
 
-        var server = ignite.server();
+    var server = ignite.server();
 
-        var host = server.host();
+    var host = server.host();
 
-        assert.ok(host.indexOf('127.0.0.1') !== -1, "Incorrect host.");
+    assert.ok(host.indexOf('127.0.0.1') !== -1, "Incorrect host.");
 
-        TestUtils.testDone();
-    }
+    TestUtils.testDone();
+  }
 }
 
 ignitionNotStartWithSeveralPorts = function() {
-    Ignition.start(['127.0.0.1:9090...9100'], null, onConnect);
+  Ignition.start(['127.0.0.1:9090...9100'], null, onConnect);
 
-    function onConnect(error, ignite) {
-        if (error) {
-            var assert = require("assert");
+  function onConnect(error, ignite) {
+    if (error) {
+      assert.ok(error.indexOf("Incorrect address format") !== -1, "Incorrect message.")
 
-            assert.ok(error.indexOf("Incorrect address format") !== -1, "Incorrect message.")
+      TestUtils.testDone();
 
-            TestUtils.testDone();
-
-            return;
-        }
-
-        TestUtils.testFails("Exception should be thrown.");
+      return;
     }
+
+    TestUtils.testFails("Exception should be thrown.");
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d7e54650/modules/nodejs/src/test/js/test-key.js
----------------------------------------------------------------------
diff --git a/modules/nodejs/src/test/js/test-key.js b/modules/nodejs/src/test/js/test-key.js
index b39aa51..b02a6b5 100644
--- a/modules/nodejs/src/test/js/test-key.js
+++ b/modules/nodejs/src/test/js/test-key.js
@@ -17,10 +17,6 @@
 
 var TestUtils = require("./test-utils").TestUtils;
 
-var Apache = require(TestUtils.scriptPath());
-var Cache = Apache.Cache;
-var Server = Apache.Server;
-
 var assert = require("assert");
 
 testStartWithoutKey = function() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d7e54650/modules/nodejs/src/test/js/test-node.xml
----------------------------------------------------------------------
diff --git a/modules/nodejs/src/test/js/test-node.xml b/modules/nodejs/src/test/js/test-node.xml
deleted file mode 100644
index a1a17e4..0000000
--- a/modules/nodejs/src/test/js/test-node.xml
+++ /dev/null
@@ -1,97 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-
-<!--
-  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.
--->
-
-<!--
-    Ignite configuration with all defaults and enabled p2p deployment and enabled events.
--->
-<beans xmlns="http://www.springframework.org/schema/beans"
-       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-       xmlns:util="http://www.springframework.org/schema/util"
-       xsi:schemaLocation="
-        http://www.springframework.org/schema/beans
-        http://www.springframework.org/schema/beans/spring-beans.xsd
-        http://www.springframework.org/schema/util
-        http://www.springframework.org/schema/util/spring-util.xsd">
-    <bean id="ignite.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
-        <!-- Set to true to enable distributed class loading for examples, default is false. -->
-        <property name="peerClassLoadingEnabled" value="true"/>
-
-        <property name="cacheConfiguration">
-            <list>
-                <!-- Partitioned cache example configuration (Atomic mode). -->
-                <bean class="org.apache.ignite.configuration.CacheConfiguration">
-                    <property name="name" value="mycache"/>
-                    <property name="atomicityMode" value="ATOMIC"/>
-                    <property name="backups" value="1"/>
-                </bean>
-            </list>
-        </property>
-
-        <property name="marshaller">
-            <bean class="org.apache.ignite.marshaller.optimized.OptimizedMarshaller">
-                <!-- Set to false to allow non-serializable objects in examples, default is true. -->
-                <property name="requireSerializable" value="false"/>
-            </bean>
-        </property>
-
-        <property name="connectorConfiguration">
-            <bean class="org.apache.ignite.configuration.ConnectorConfiguration">
-                <property name="jettyPath" value="rest-jetty.xml"/>
-            </bean>
-        </property>
-
-        <!-- Enable task execution events for examples. -->
-        <property name="includeEventTypes">
-            <list>
-                <!--Task execution events-->
-                <util:constant static-field="org.apache.ignite.events.EventType.EVT_TASK_STARTED"/>
-                <util:constant static-field="org.apache.ignite.events.EventType.EVT_TASK_FINISHED"/>
-                <util:constant static-field="org.apache.ignite.events.EventType.EVT_TASK_FAILED"/>
-                <util:constant static-field="org.apache.ignite.events.EventType.EVT_TASK_TIMEDOUT"/>
-                <util:constant static-field="org.apache.ignite.events.EventType.EVT_TASK_SESSION_ATTR_SET"/>
-                <util:constant static-field="org.apache.ignite.events.EventType.EVT_TASK_REDUCED"/>
-
-                <!--Cache events-->
-                <util:constant static-field="org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_PUT"/>
-                <util:constant static-field="org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_READ"/>
-                <util:constant static-field="org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_REMOVED"/>
-            </list>
-        </property>
-
-        <!-- Explicitly configure TCP discovery SPI to provide list of initial nodes. -->
-        <property name="discoverySpi">
-            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
-                <property name="ipFinder">
-                    <!--
-                        Ignite provides several options for automatic discovery that can be used
-                        instead os static IP based discovery. For information on all options refer
-                        to our documentation: http://apacheignite.readme.io/docs/cluster-config
-                    -->
-                    <!-- Uncomment static IP finder to enable static-based discovery of initial nodes. -->
-                    <!--<bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">-->
-                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
-                        <constructor-arg>
-                            <value>true</value>
-                        </constructor-arg>
-                    </bean>
-                </property>
-            </bean>
-        </property>
-    </bean>
-</beans>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d7e54650/modules/nodejs/src/test/js/test-runner.js
----------------------------------------------------------------------
diff --git a/modules/nodejs/src/test/js/test-runner.js b/modules/nodejs/src/test/js/test-runner.js
index 471df11..9ee1092 100644
--- a/modules/nodejs/src/test/js/test-runner.js
+++ b/modules/nodejs/src/test/js/test-runner.js
@@ -39,6 +39,7 @@ TestRunner.runTest = function() {
         console.log("node js test failed: function with name " + functionName + " not found");
         return;
     }
+
     global[functionName]();
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d7e54650/modules/nodejs/src/test/js/test-utils.js
----------------------------------------------------------------------
diff --git a/modules/nodejs/src/test/js/test-utils.js b/modules/nodejs/src/test/js/test-utils.js
index ee83d7f..ae2f0f0 100644
--- a/modules/nodejs/src/test/js/test-utils.js
+++ b/modules/nodejs/src/test/js/test-utils.js
@@ -59,16 +59,20 @@ TestUtils.classpath = function(dir) {
   function walk(dir, done) {
     var results = [];
     var list = fs.readdirSync(dir)
+
     for (var i = 0; i < list.length; ++i) {
       file = path.resolve(dir, list[i]);
+
       var stat = fs.statSync(file);
+
       if (stat && stat.isDirectory()) {
         if (list[i] != "optional" && file.indexOf("optional") !== -1 && file.indexOf("rest") == -1 )
           continue;
 
         var sublist = walk(file);
         results = results.concat(sublist);
-      } else {
+      }
+      else {
         if (file.indexOf(".jar") !== -1) {
           results.push(file);
         }


[14/20] incubator-ignite git commit: #ignite-965: code style compute.js

Posted by sb...@apache.org.
#ignite-965: code style compute.js


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

Branch: refs/heads/ignite-965
Commit: da3e887b6f20ac7374f3c32438289942c7385565
Parents: 905a3e6
Author: ivasilinets <iv...@gridgain.com>
Authored: Tue Jun 23 20:20:50 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Tue Jun 23 20:20:50 2015 +0300

----------------------------------------------------------------------
 modules/nodejs/src/main/js/compute.js | 46 ++++++++++++++----------------
 1 file changed, 21 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/da3e887b/modules/nodejs/src/main/js/compute.js
----------------------------------------------------------------------
diff --git a/modules/nodejs/src/main/js/compute.js b/modules/nodejs/src/main/js/compute.js
index 00f7b54..bce737a 100644
--- a/modules/nodejs/src/main/js/compute.js
+++ b/modules/nodejs/src/main/js/compute.js
@@ -20,23 +20,18 @@ var Server = require("./server").Server;
 /**
  * @constructor
  * @this {Compute}
- * @param {Server} server Server class
+ * @param {Server} server Server
  */
 function Compute(server) {
   this._server = server;
 }
 
 /**
- * Callback for affinityRun
- * @callback Compute~runnable
- */
-
-/**
  * @this {Compute}
- * @param {string} cacheName Cache name.
- * @param {string} key Key.
- * @param {Compute~runnable} runnable Function without parameters
- * @param {Cache~noValue} callback Callback
+ * @param {string} cacheName Cache name
+ * @param {string} key Key
+ * @param runnable Function without parameters and return value
+ * @param {noValue} callback Callback
  */
 Compute.prototype.affinityRun = function(cacheName, key, runnable, callback) {
   this._server.runCommand("affscriptrun", [Server.pair("cacheName", cacheName),
@@ -45,10 +40,10 @@ Compute.prototype.affinityRun = function(cacheName, key, runnable, callback) {
 
 /**
  * @this {Compute}
- * @param {string} cacheName Cache name.
- * @param {string} key Key.
- * @param {Compute~runnable} runnable Function without parameters
- * @param {Cache~onGet} callback Callback
+ * @param {string} cacheName Cache name
+ * @param {string} key Key
+ * @param runnable Function without parameters
+ * @param {onGet} callback Callback
  */
 Compute.prototype.affinityCall = function(cacheName, key, runnable, callback) {
   this._server.runCommand("affscriptcall", [Server.pair("cacheName", cacheName),
@@ -56,29 +51,30 @@ Compute.prototype.affinityCall = function(cacheName, key, runnable, callback) {
 }
 
 /**
- * @param{Cache~noValue} f Function
+ * @param {noValue} f Function
  * @returns {string} Encoding function
  */
 Compute.prototype._escape = function(f) {
-  var f = f.toString();
   var qs = require('querystring');
-  return qs.escape(f);
+
+  return qs.escape(f.toString());
 }
 
 /**
  * @this {Compute}
- * @param {ComputeTask} task Compute task
- * @param {string} arg  Argument
- * @param {} callback Callback
+ * @param {MapFunction} map Map function
+ * @param {ReduceFunction} reduce Reduce function
+ * @param {string} arg Argument
+ * @param {onGet} callback Callback
  */
 Compute.prototype.execute = function(map, reduce, arg, callback) {
-   var params = [];
+  var params = [];
 
-    params.push(Server.pair("map", this._escape(map)));
-    params.push(Server.pair("reduce", this._escape(reduce)));
-    params.push(Server.pair("arg", this._escape(arg)));
+  params.push(Server.pair("map", this._escape(map)));
+  params.push(Server.pair("reduce", this._escape(reduce)));
+  params.push(Server.pair("arg", this._escape(arg)));
 
-    this._server.runCommand("execscripttask", params, callback);
+  this._server.runCommand("execscripttask", params, callback);
 }
 
 /**


[03/20] incubator-ignite git commit: #ignite-965: code style GridJettyRestHandler.

Posted by sb...@apache.org.
#ignite-965: code style GridJettyRestHandler.


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

Branch: refs/heads/ignite-965
Commit: d24ba8452aa94c2b74d21b4cfe68ff409fd5d56a
Parents: 0a257f1
Author: ivasilinets <iv...@gridgain.com>
Authored: Tue Jun 23 18:29:48 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Tue Jun 23 18:29:48 2015 +0300

----------------------------------------------------------------------
 .../rest/protocols/http/jetty/GridJettyRestHandler.java         | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d24ba845/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 7d71530..249b9ca 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
@@ -456,6 +456,7 @@ public class GridJettyRestHandler extends AbstractHandler {
                 restReq0.key(params.get("key"));
 
                 restReq = restReq0;
+
                 break;
             }
 
@@ -463,8 +464,8 @@ public class GridJettyRestHandler extends AbstractHandler {
                 RestComputeTaskRequest restReq0 = new RestComputeTaskRequest();
 
                 restReq0.mapFunc((String)params.get("map"));
-                restReq0.argument((String) params.get("arg"));
-                restReq0.reduceFunc((String) params.get("reduce"));
+                restReq0.argument((String)params.get("arg"));
+                restReq0.reduceFunc((String)params.get("reduce"));
 
                 restReq = restReq0;
 


[11/20] incubator-ignite git commit: #ignite-965: code style GridJavaProcess.

Posted by sb...@apache.org.
#ignite-965: code style GridJavaProcess.


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

Branch: refs/heads/ignite-965
Commit: 53ed5c0ed21bc072cc9b58b8be8153c9813b68b7
Parents: d67b15a
Author: ivasilinets <iv...@gridgain.com>
Authored: Tue Jun 23 19:58:49 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Tue Jun 23 19:58:49 2015 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/internal/util/GridJavaProcess.java  | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/53ed5c0e/modules/core/src/main/java/org/apache/ignite/internal/util/GridJavaProcess.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/GridJavaProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/util/GridJavaProcess.java
index a854f82..164a295 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/GridJavaProcess.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/GridJavaProcess.java
@@ -152,7 +152,7 @@ public final class GridJavaProcess {
 
         builder.redirectErrorStream(true);
 
-        return setGrabber(builder, log, printC, procKilledC);
+        return startProcess(builder, log, printC, procKilledC);
     }
 
     /**
@@ -175,7 +175,7 @@ public final class GridJavaProcess {
 
         builder.environment().putAll(env);
 
-        return setGrabber(builder, log, printC, procKilledC);
+        return startProcess(builder, log, printC, procKilledC);
     }
 
 
@@ -187,7 +187,7 @@ public final class GridJavaProcess {
      * @return Wrapper around {@link Process}
      * @throws Exception If any problem occurred.
      */
-    private static GridJavaProcess setGrabber(ProcessBuilder builder,
+    private static GridJavaProcess startProcess(ProcessBuilder builder,
         @Nullable IgniteLogger log, @Nullable IgniteInClosure<String> printC,
         @Nullable GridAbsClosure procKilledC) throws Exception {
         if (!(U.isLinux() || U.isMacOs() || U.isWindows()))


[10/20] incubator-ignite git commit: #ignite-965: code style GridJavaProcess.

Posted by sb...@apache.org.
#ignite-965: code style GridJavaProcess.


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

Branch: refs/heads/ignite-965
Commit: d67b15a2fdb25045c005f752bb55422fdb481312
Parents: a28b02d
Author: ivasilinets <iv...@gridgain.com>
Authored: Tue Jun 23 19:57:30 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Tue Jun 23 19:57:30 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/util/GridJavaProcess.java   | 46 +++++++++-----------
 1 file changed, 21 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d67b15a2/modules/core/src/main/java/org/apache/ignite/internal/util/GridJavaProcess.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/GridJavaProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/util/GridJavaProcess.java
index 693d486..a854f82 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/GridJavaProcess.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/GridJavaProcess.java
@@ -120,14 +120,6 @@ public final class GridJavaProcess {
     public static GridJavaProcess exec(String clsName, String params, @Nullable IgniteLogger log,
         @Nullable IgniteInClosure<String> printC, @Nullable GridAbsClosure procKilledC,
         @Nullable Collection<String> jvmArgs, @Nullable String cp) throws Exception {
-        if (!(U.isLinux() || U.isMacOs() || U.isWindows()))
-            throw new Exception("Your OS is not supported.");
-
-        GridJavaProcess gjProc = new GridJavaProcess();
-
-        gjProc.log = log;
-        gjProc.procKilledC = procKilledC;
-
         List<String> procParams = params == null || params.isEmpty() ?
             Collections.<String>emptyList() : Arrays.asList(params.split(" "));
 
@@ -160,17 +152,7 @@ public final class GridJavaProcess {
 
         builder.redirectErrorStream(true);
 
-        Process proc = builder.start();
-
-        gjProc.osGrabber = gjProc.new ProcessStreamGrabber(proc.getInputStream(), printC);
-        gjProc.esGrabber = gjProc.new ProcessStreamGrabber(proc.getErrorStream(), printC);
-
-        gjProc.osGrabber.start();
-        gjProc.esGrabber.start();
-
-        gjProc.proc = proc;
-
-        return gjProc;
+        return setGrabber(builder, log, printC, procKilledC);
     }
 
     /**
@@ -187,7 +169,27 @@ public final class GridJavaProcess {
     public static GridJavaProcess exec(List<String> cmd, Map<String, String> env, @Nullable IgniteLogger log,
         @Nullable IgniteInClosure<String> printC, @Nullable GridAbsClosure procKilledC)
         throws Exception {
+        ProcessBuilder builder = new ProcessBuilder(cmd);
 
+        builder.redirectErrorStream(true);
+
+        builder.environment().putAll(env);
+
+        return setGrabber(builder, log, printC, procKilledC);
+    }
+
+
+    /**
+     * @param builder Process builder.
+     * @param log Log to use.
+     * @param printC Optional closure to be called each time wrapped process prints line to system.out or system.err.
+     * @param procKilledC Optional closure to be called when process termination is detected.
+     * @return Wrapper around {@link Process}
+     * @throws Exception If any problem occurred.
+     */
+    private static GridJavaProcess setGrabber(ProcessBuilder builder,
+        @Nullable IgniteLogger log, @Nullable IgniteInClosure<String> printC,
+        @Nullable GridAbsClosure procKilledC) throws Exception {
         if (!(U.isLinux() || U.isMacOs() || U.isWindows()))
             throw new Exception("Your OS is not supported.");
 
@@ -196,12 +198,6 @@ public final class GridJavaProcess {
         gjProc.log = log;
         gjProc.procKilledC = procKilledC;
 
-        ProcessBuilder builder = new ProcessBuilder(cmd);
-
-        builder.redirectErrorStream(true);
-
-        builder.environment().putAll(env);
-
         Process proc = builder.start();
 
         gjProc.osGrabber = gjProc.new ProcessStreamGrabber(proc.getInputStream(), printC);


[17/20] incubator-ignite git commit: #ignite-965: code style server.js

Posted by sb...@apache.org.
#ignite-965: code style server.js


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

Branch: refs/heads/ignite-965
Commit: 094b60529a4e88c6185bee7d1fb10c37196810b3
Parents: 294531b
Author: ivasilinets <iv...@gridgain.com>
Authored: Tue Jun 23 20:24:13 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Tue Jun 23 20:24:13 2015 +0300

----------------------------------------------------------------------
 modules/nodejs/src/main/js/server.js | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/094b6052/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 910b7c3..f1c9d92 100644
--- a/modules/nodejs/src/main/js/server.js
+++ b/modules/nodejs/src/main/js/server.js
@@ -117,7 +117,6 @@ Server.prototype.runCommand = function(cmdName, params, callback) {
     });
   }
 
-  console.log("!!!!!!!!!!!!!!!!!REQUEST " + options.path);
   var request = http.request(options, streamCallback);
 
   request.setTimeout(5000, callback.bind(null, "Request timeout: >5 sec"));


[08/20] incubator-ignite git commit: #ignite-965: code style RestComputeTaskRequest.

Posted by sb...@apache.org.
#ignite-965: code style RestComputeTaskRequest.


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

Branch: refs/heads/ignite-965
Commit: dbc29fdec5e79012d6046d38d02d8e229d27e84c
Parents: 8268791
Author: ivasilinets <iv...@gridgain.com>
Authored: Tue Jun 23 19:46:25 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Tue Jun 23 19:46:25 2015 +0300

----------------------------------------------------------------------
 .../IgniteComputeTaskCommandHandler.java        |  2 +-
 .../rest/request/RestComputeTaskRequest.java    | 44 +++++++++++++-------
 .../http/jetty/GridJettyRestHandler.java        |  4 +-
 3 files changed, 32 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dbc29fde/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/compute/IgniteComputeTaskCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/compute/IgniteComputeTaskCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/compute/IgniteComputeTaskCommandHandler.java
index 59feae7..4112ff8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/compute/IgniteComputeTaskCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/compute/IgniteComputeTaskCommandHandler.java
@@ -63,7 +63,7 @@ public class IgniteComputeTaskCommandHandler extends GridRestCommandHandlerAdapt
         final RestComputeTaskRequest req0 = (RestComputeTaskRequest) req;
 
         Object execRes = ctx.grid().compute().execute(
-            new JsTask(req0.mapFunc(), req0.argument(), req0.reduceFunc(), ctx), null);
+            new JsTask(req0.mapFunction(), req0.argument(), req0.reduceFunction(), ctx), null);
 
         return new GridFinishedFuture<>(new GridRestResponse(execRes));
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dbc29fde/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/RestComputeTaskRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/RestComputeTaskRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/RestComputeTaskRequest.java
index 89e04da..cf5c7c0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/RestComputeTaskRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/RestComputeTaskRequest.java
@@ -17,10 +17,6 @@
 
 package org.apache.ignite.internal.processors.rest.request;
 
-import org.apache.ignite.internal.util.typedef.T3;
-
-import java.util.*;
-
 /**
  * Compute task request.
  */
@@ -31,30 +27,48 @@ public class RestComputeTaskRequest extends GridRestRequest {
     /** Function argument. */
     private String arg;
 
+    /** Reduce function. */
     private String reduceFunc;
 
-    public void reduceFunc(String reduceFunc) {
+    /**
+     * @param reduceFunc Reduce function.
+     */
+    public void reduceFunction(String reduceFunc) {
         this.reduceFunc = reduceFunc;
     }
 
-    public String reduceFunc() {
-
+    /**
+     * @return Reduce function.
+     */
+    public String reduceFunction() {
         return reduceFunc;
     }
 
-    public String mapFunc() {
-        return mapFunc;
-    }
-
-    public String argument() {
-        return arg;
+    /**
+     * @param mapFunc Map function.
+     */
+    public void mapFunction(String mapFunc) {
+        this.mapFunc = mapFunc;
     }
 
-    public void mapFunc(String mapFunc) {
-        this.mapFunc = mapFunc;
+    /**
+     * @return Map function.
+     */
+    public String mapFunction() {
+        return mapFunc;
     }
 
+    /**
+     * @param arg Argument.
+     */
     public void argument(String arg) {
         this.arg = arg;
     }
+
+    /**
+     * @return Argument.
+     */
+    public String argument() {
+        return arg;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dbc29fde/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 249b9ca..0d961be 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
@@ -463,9 +463,9 @@ public class GridJettyRestHandler extends AbstractHandler {
             case EXECUTE_TASK: {
                 RestComputeTaskRequest restReq0 = new RestComputeTaskRequest();
 
-                restReq0.mapFunc((String)params.get("map"));
+                restReq0.mapFunction((String) params.get("map"));
                 restReq0.argument((String)params.get("arg"));
-                restReq0.reduceFunc((String)params.get("reduce"));
+                restReq0.reduceFunction((String) params.get("reduce"));
 
                 restReq = restReq0;
 


[05/20] incubator-ignite git commit: #ignite-965: codestyle IgniteComputeTaskCommandHandler.

Posted by sb...@apache.org.
#ignite-965: codestyle IgniteComputeTaskCommandHandler.


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

Branch: refs/heads/ignite-965
Commit: 57e9845a56ebba443b83f42c2348f6998e511ba3
Parents: 30ec9f7
Author: ivasilinets <iv...@gridgain.com>
Authored: Tue Jun 23 19:40:14 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Tue Jun 23 19:40:14 2015 +0300

----------------------------------------------------------------------
 .../IgniteComputeTaskCommandHandler.java        | 100 ++++++++++---------
 .../scripting/IgniteScriptProcessor.java        |   2 +-
 modules/nodejs/src/main/js/compute.js           |  10 --
 3 files changed, 52 insertions(+), 60 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/57e9845a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/compute/IgniteComputeTaskCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/compute/IgniteComputeTaskCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/compute/IgniteComputeTaskCommandHandler.java
index 33f4893..33cd45b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/compute/IgniteComputeTaskCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/compute/IgniteComputeTaskCommandHandler.java
@@ -56,29 +56,39 @@ public class IgniteComputeTaskCommandHandler extends GridRestCommandHandlerAdapt
     @Override public IgniteInternalFuture<GridRestResponse> handleAsync(GridRestRequest req) {
         assert req != null;
 
-        assert req instanceof RestComputeTaskRequest : "Invalid type of compute request.";
+        assert req instanceof RestComputeTaskRequest : "Invalid type of compute task request.";
 
         assert SUPPORTED_COMMANDS.contains(req.command());
 
         final RestComputeTaskRequest req0 = (RestComputeTaskRequest) req;
 
-        Object res = ctx.grid().compute().execute(new JsTask(req0.mapFunc(), req0.argument(), req0.reduceFunc(), ctx), null);
+        Object execRes = ctx.grid().compute().execute(
+            new JsTask(req0.mapFunc(), req0.argument(), req0.reduceFunc(), ctx), null);
 
-        return new GridFinishedFuture<>(new GridRestResponse(res));
+        return new GridFinishedFuture<>(new GridRestResponse(execRes));
     }
 
+    /**
+     * JS Compute Task.
+     */
     private static class JsTask extends ComputeTaskAdapter<String, Object> {
         /** Mapping function. */
         private String mapFunc;
 
+        /** Reduce function. */
         private String reduceFunc;
 
-        /** Grid kernal context. */
+        /** Kernal context. */
         private GridKernalContext ctx;
 
+        /** Map function argument. */
         private String arg;
 
         /**
+         * @param mapFunc Map function.
+         * @param arg Map function argument.
+         * @param reduceFunc Reduce function.
+         * @param ctx Kernal context.
          */
         public JsTask(String mapFunc, String arg, String reduceFunc, GridKernalContext ctx) {
             this.mapFunc = mapFunc;
@@ -91,61 +101,30 @@ public class IgniteComputeTaskCommandHandler extends GridRestCommandHandlerAdapt
         @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> nodes, String arg) {
             Map<ComputeJob, ClusterNode> map = new HashMap<>();
 
-            String nodesIds = "[";
-
-            for (ClusterNode node : nodes)
-                nodesIds += "\""  + node.id().toString() + "\"" + ",";
-
-            nodesIds = nodesIds.substring(0, nodesIds.length() - 1) + "]";
-
-            String newMap = new String("function () {\n" +
-                "   var res = [];\n" +
-                "   var resCont = function(f, args, nodeId) {\n" +
-                "       res.push([f.toString(), args, nodeId])\n" +
-                "   }\n" +
-                "   var locF = " + mapFunc + "; \n locF(" +
-                    nodesIds + ", " +
-                "\"" + this.arg + "\"" +
-                ", resCont.bind(null)" + ");\n" +
-                "   return res;\n" +
-                "}");
-
-            List mapRes = (List)ctx.scripting().runJSFunction(newMap);
-
-            for (Object arr : mapRes) {
-                Object[] nodeTask = ((List)arr).toArray();
-
-                final String func = (String)nodeTask[0];
-
-                final List argv = (List) nodeTask[1];
+            List jsMapResult = (List)ctx.scripting().runJSFunction(wrapMapperFunction(nodes));
 
-                String nodeIdStr = (String) nodeTask[2];
+            for (Object jobMapping : jsMapResult) {
+                List task = (List)jobMapping;
 
-                UUID nodeId = UUID.fromString(nodeIdStr);
+                final String func = (String)task.get(0);
+                final List argv = (List)task.get(1);
+                String nodeId = (String)task.get(2);
 
-                ClusterNode node = ctx.grid().cluster().node(nodeId);
+                ClusterNode node = ctx.grid().cluster().node(UUID.fromString(nodeId));
 
                 map.put(new ComputeJobAdapter() {
-                    /** Ignite. */
                     @IgniteInstanceResource
                     private Ignite ignite;
 
                     @Override public Object execute() throws IgniteException {
-                        System.out.println("Compute job on node " + ignite.cluster().localNode().id());
-                        try {
-                            String[] argv1 = new String[argv.size()];
-
-                            for (int i = 0; i < argv1.length; ++i)
-                                argv1[i] = "\"" + argv.get(i).toString() + "\"";
-
-                            return ctx.scripting().runJSFunction(func, argv1);
-                        }
-                        catch (Exception e) {
-                            throw new IgniteException(e);
-                        }
+                        String[] argv1 = new String[argv.size()];
+
+                        for (int i = 0; i < argv1.length; ++i)
+                            argv1[i] = "\"" + argv.get(i).toString() + "\"";
+
+                        return ctx.scripting().runJSFunction(func, argv1);
                     }
                 }, node);
-
             }
 
             return map;
@@ -158,7 +137,30 @@ public class IgniteComputeTaskCommandHandler extends GridRestCommandHandlerAdapt
             for (ComputeJobResult res : results)
                 data.add(res.getData());
 
-            return ctx.scripting().runJSFunction(reduceFunc, new String[]{data.toString()});
+            return ctx.scripting().runJSFunction(reduceFunc, data.toString());
+        }
+
+        /**
+         * @param nodes Cluster nodes.
+         * @return Script running map function.
+         */
+        private String wrapMapperFunction(List<ClusterNode> nodes) {
+            List<String> ids = new ArrayList<>();
+
+            for (ClusterNode node : nodes)
+                ids.add("\"" + node.id().toString() + "\"");
+
+            String sep = System.getProperty("line.separator");
+
+            return "function () {" + sep +
+                "       var res = [];" + sep +
+                "       var emitFunc = function(f, args, nodeId) {" + sep +
+                "           res.push([f.toString(), args, nodeId])" + sep +
+                "       }" + sep +
+                "       var f = " + mapFunc + ";" + sep +
+                "       f(" + ids + ", " + "\"" + this.arg + "\"" + ", emitFunc.bind(null)" + ");" + sep +
+                "       return res;" + sep +
+                "   }";
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/57e9845a/modules/core/src/main/java/org/apache/ignite/internal/processors/scripting/IgniteScriptProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/scripting/IgniteScriptProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/scripting/IgniteScriptProcessor.java
index 094373c..1a72920 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/scripting/IgniteScriptProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/scripting/IgniteScriptProcessor.java
@@ -61,7 +61,7 @@ public class IgniteScriptProcessor extends GridProcessorAdapter {
      * @return Script result.
      * @throws ScriptException If script failed.
      */
-    public Object runJSFunction(String script, String[] args) throws IgniteException {
+    public Object runJSFunction(String script, String... args) throws IgniteException {
         try {
             return jsEngine.eval(callJsFunction(script, args));
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/57e9845a/modules/nodejs/src/main/js/compute.js
----------------------------------------------------------------------
diff --git a/modules/nodejs/src/main/js/compute.js b/modules/nodejs/src/main/js/compute.js
index 7f45f5b..e452847 100644
--- a/modules/nodejs/src/main/js/compute.js
+++ b/modules/nodejs/src/main/js/compute.js
@@ -104,14 +104,4 @@ Compute.prototype.execute = function(map, reduce, arg, callback) {
  * @returns {string} Result
  */
 
-/**
- * @this{Compute}
- * @param {MapFunction} map Map function
- * @param {ReduceFunction} reduce Reduce function
- * @param {string} arg Argument
- * @param {onGet} callback Callback
- */
-Compute.prototype._mapWrapper = function(map) {
-
-}
 exports.Compute = Compute


[07/20] incubator-ignite git commit: #ignite-965: code style IgniteComputeCommandHandler.

Posted by sb...@apache.org.
#ignite-965: code style IgniteComputeCommandHandler.


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

Branch: refs/heads/ignite-965
Commit: 8268791bf4ec34daaee966c72a55f047e750b626
Parents: 5e60562
Author: ivasilinets <iv...@gridgain.com>
Authored: Tue Jun 23 19:42:00 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Tue Jun 23 19:42:00 2015 +0300

----------------------------------------------------------------------
 .../rest/handlers/compute/IgniteComputeCommandHandler.java   | 8 ++++----
 1 file changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8268791b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/compute/IgniteComputeCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/compute/IgniteComputeCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/compute/IgniteComputeCommandHandler.java
index acb4ab6..f36e024 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/compute/IgniteComputeCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/compute/IgniteComputeCommandHandler.java
@@ -27,7 +27,6 @@ import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.resources.*;
 
-import javax.script.*;
 import java.util.*;
 
 import static org.apache.ignite.internal.processors.rest.GridRestCommand.*;
@@ -37,7 +36,8 @@ import static org.apache.ignite.internal.processors.rest.GridRestCommand.*;
  */
 public class IgniteComputeCommandHandler extends GridRestCommandHandlerAdapter {
     /** Supported commands. */
-    private static final Collection<GridRestCommand> SUPPORTED_COMMANDS = U.sealList(AFFINITY_RUN,
+    private static final Collection<GridRestCommand> SUPPORTED_COMMANDS = U.sealList(
+        AFFINITY_RUN,
         AFFINITY_CALL);
 
     /**
@@ -73,7 +73,7 @@ public class IgniteComputeCommandHandler extends GridRestCommandHandlerAdapter {
                     }
                 });
 
-                return new GridFinishedFuture<>(new GridRestResponse("AFFINITY RUN " + req));
+                return new GridFinishedFuture<>(new GridRestResponse());
 
             case AFFINITY_CALL:
                 Object res = ctx.grid().compute().affinityCall(req0.cacheName(), req0.key(), new IgniteCallable<Object>() {
@@ -85,7 +85,7 @@ public class IgniteComputeCommandHandler extends GridRestCommandHandlerAdapter {
                     }
                 });
 
-                return new GridFinishedFuture<>(new GridRestResponse("AFFINITY RUN " + res));
+                return new GridFinishedFuture<>(new GridRestResponse(res));
         }
 
         return new GridFinishedFuture<>();


[18/20] incubator-ignite git commit: #ignite-965: code style test-cache-api.js

Posted by sb...@apache.org.
#ignite-965: code style test-cache-api.js


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

Branch: refs/heads/ignite-965
Commit: e7375e936d08eee4c88ecafec2a995a5527075d4
Parents: 094b605
Author: ivasilinets <iv...@gridgain.com>
Authored: Tue Jun 23 20:28:00 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Tue Jun 23 20:28:00 2015 +0300

----------------------------------------------------------------------
 .../nodejs/src/test/js/simple-compute-task.js   |  61 ----------
 modules/nodejs/src/test/js/test-cache-api.js    |   4 -
 modules/nodejs/src/test/js/test.js              | 116 -------------------
 3 files changed, 181 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e7375e93/modules/nodejs/src/test/js/simple-compute-task.js
----------------------------------------------------------------------
diff --git a/modules/nodejs/src/test/js/simple-compute-task.js b/modules/nodejs/src/test/js/simple-compute-task.js
deleted file mode 100644
index d7eb91d..0000000
--- a/modules/nodejs/src/test/js/simple-compute-task.js
+++ /dev/null
@@ -1,61 +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.
- */
-
-function CharacterCountTask() {
-}
-
-CharacterCountTask.prototype.map = function(nodes, arg) {
-  var words = arg.split(" ");
-
-  var results = [];
-
-  var nodeId = 0;
-
-  function compute(args) {
-    println(">>> Printing " + args);
-
-    return args[0].length;
-  }
-
-
-  for (var word of words) {
-    var node = nodes[nodeId];
-
-    if (nodeId < nodes.length - 1) {
-      nodeId++;
-    }
-
-    var TestUtils = require("./test-utils").TestUtils;
-    var Apache = require(TestUtils.scriptPath());
-    var ComputeJob = Apache.ComputeJob;
-    results.push(new ComputeJob(compute, [word], node));
-  }
-
-  return results;
-}
-
-CharacterCountTask.prototype.reduce = function(results) {
-  var sum = 0;
-
-  for (var res of results) {
-    sum += parseInt(res, 10);
-  }
-
-  return sum;
-}
-
-exports.CharacterCountTask = CharacterCountTask

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e7375e93/modules/nodejs/src/test/js/test-cache-api.js
----------------------------------------------------------------------
diff --git a/modules/nodejs/src/test/js/test-cache-api.js b/modules/nodejs/src/test/js/test-cache-api.js
index 48ff011..1101493 100644
--- a/modules/nodejs/src/test/js/test-cache-api.js
+++ b/modules/nodejs/src/test/js/test-cache-api.js
@@ -17,10 +17,6 @@
 
 var TestUtils = require("./test-utils").TestUtils;
 
-var Apache = require(TestUtils.scriptPath());
-var Cache = Apache.Cache;
-var Server = Apache.Server;
-
 var assert = require("assert");
 
 testPutGet = function() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e7375e93/modules/nodejs/src/test/js/test.js
----------------------------------------------------------------------
diff --git a/modules/nodejs/src/test/js/test.js b/modules/nodejs/src/test/js/test.js
deleted file mode 100644
index 7485072..0000000
--- a/modules/nodejs/src/test/js/test.js
+++ /dev/null
@@ -1,116 +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.
- */
-
-module.exports = {
-    'Test put/get' : function(test) {
-        test.expect(1);
-
-        var TestUtils = require("./test_utils").TestUtils;
-        var Cache = require(TestUtils.scriptPath() + "cache").Cache;
-        var Server = require(TestUtils.scriptPath() + "server").Server;
-
-        var assert = require('assert');
-
-        var node = startIgniteNode();
-
-        setTimeout(initCache, 10000);
-
-        function initCache() {
-            var server = new Server('127.0.0.1', 9090);
-            var cache = new Cache(server, "mycache");
-            cache.put("mykey", "6", onPut.bind(null, cache));
-        }
-
-        function onPut(cache, error) {
-            if (error) {
-                console.error("Failed to put " + error);
-                finishTest(test, node);
-                return;
-            }
-
-            console.log("Put finished");
-            cache.get("mykey", onGet);
-        }
-
-        function onGet(error, value) {
-            if (error) {
-                console.error("Failed to get " + error);
-                finishTest(test, node);
-                return;
-            }
-
-            console.log("Get finished");
-            test.ok(value === "6", "This shouldn't fail " + value + "<>6");
-            finishTest(test, node);
-        }
-    },
-    'Test connection' : function(test) {
-        test.expect(0);
-
-        var node = startIgniteNode();
-        var TestUtils = require("./test_utils").TestUtils;
-        var Server = require(TestUtils.scriptPath() + "server").Server;
-
-        setTimeout(initServer, 10000);
-
-        function initServer() {
-            var server = new Server('127.0.0.1', 9090);
-
-            console.log("Try to check connection");
-
-            server.checkConnection(onConnect);
-        }
-
-        function onConnect(error) {
-            if (error) {
-                finishWithError(test/*, node*/, error);
-                return;
-            }
-            console.log("Successfully connected");
-            finishTest(test, node);
-        }
-    },
-    'Test ignition' : function(test) {
-        test.expect(1);
-
-        var node = startIgniteNode('127.0.0.1', 9090);
-        var TestUtils = require("./test_utils").TestUtils;
-        var Ignition = require(TestUtils.scriptPath() + "ignition").Ignition;
-
-        setTimeout(Ignition.start.bind(null, 9090, ['127.0.0.0', '127.0.0.1'], onConnect), 5000);
-
-        function onConnect(error, server) {
-            if (error) {
-                finishWithError(test, node, error);
-                return;
-            }
-            test.ok(server.host() === '127.0.0.1')
-            finishTest(test, node);
-        }
-    }
- };
-
-function finishWithError(test, node, error) {
-    console.log("Error: " + error);
-    test.ok(false);
-    finishTest(test, node);
-}
-
-function finishTest(test, node) {
-    node.kill();
-    test.done();
-}
\ No newline at end of file


[09/20] incubator-ignite git commit: #ignite-965: code style GridRestCommand.

Posted by sb...@apache.org.
#ignite-965: code style GridRestCommand.


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

Branch: refs/heads/ignite-965
Commit: a28b02d3ec8aeec97a7b66f4d772380d203bd315
Parents: dbc29fd
Author: ivasilinets <iv...@gridgain.com>
Authored: Tue Jun 23 19:48:39 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Tue Jun 23 19:48:39 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/processors/rest/GridRestCommand.java       | 6 +++---
 modules/nodejs/src/main/js/compute.js                          | 6 +++---
 2 files changed, 6 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a28b02d3/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 91022fa..f964cb4 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
@@ -97,13 +97,13 @@ public enum GridRestCommand {
     QUIT("quit"),
 
     /** Affinity run. */
-    AFFINITY_RUN("affrun"),
+    AFFINITY_RUN("affscriptrun"),
 
     /** Affinity call. */
-    AFFINITY_CALL("affcall"),
+    AFFINITY_CALL("affscriptcall"),
 
     /** Execute task. */
-    EXECUTE_TASK("exectask");
+    EXECUTE_TASK("execscripttask");
 
     /** Enum values. */
     private static final GridRestCommand[] VALS = values();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a28b02d3/modules/nodejs/src/main/js/compute.js
----------------------------------------------------------------------
diff --git a/modules/nodejs/src/main/js/compute.js b/modules/nodejs/src/main/js/compute.js
index e452847..00f7b54 100644
--- a/modules/nodejs/src/main/js/compute.js
+++ b/modules/nodejs/src/main/js/compute.js
@@ -39,7 +39,7 @@ function Compute(server) {
  * @param {Cache~noValue} callback Callback
  */
 Compute.prototype.affinityRun = function(cacheName, key, runnable, callback) {
-  this._server.runCommand("affrun", [Server.pair("cacheName", cacheName),
+  this._server.runCommand("affscriptrun", [Server.pair("cacheName", cacheName),
     Server.pair("key", key), Server.pair("func", this._escape(runnable))], callback);
 }
 
@@ -51,7 +51,7 @@ Compute.prototype.affinityRun = function(cacheName, key, runnable, callback) {
  * @param {Cache~onGet} callback Callback
  */
 Compute.prototype.affinityCall = function(cacheName, key, runnable, callback) {
-  this._server.runCommand("affcall", [Server.pair("cacheName", cacheName),
+  this._server.runCommand("affscriptcall", [Server.pair("cacheName", cacheName),
     Server.pair("key", key), Server.pair("func", this._escape(runnable))], callback);
 }
 
@@ -78,7 +78,7 @@ Compute.prototype.execute = function(map, reduce, arg, callback) {
     params.push(Server.pair("reduce", this._escape(reduce)));
     params.push(Server.pair("arg", this._escape(arg)));
 
-    this._server.runCommand("exectask", params, callback);
+    this._server.runCommand("execscripttask", params, callback);
 }
 
 /**


[15/20] incubator-ignite git commit: #ignite-965: code style compute.js

Posted by sb...@apache.org.
#ignite-965: code style compute.js


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

Branch: refs/heads/ignite-965
Commit: 4bb2004c02ac57a94af00d5001fb16a06c14e68a
Parents: da3e887
Author: ivasilinets <iv...@gridgain.com>
Authored: Tue Jun 23 20:21:10 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Tue Jun 23 20:21:10 2015 +0300

----------------------------------------------------------------------
 modules/nodejs/src/main/js/compute.js | 20 ++++++++++----------
 1 file changed, 10 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4bb2004c/modules/nodejs/src/main/js/compute.js
----------------------------------------------------------------------
diff --git a/modules/nodejs/src/main/js/compute.js b/modules/nodejs/src/main/js/compute.js
index bce737a..f00db69 100644
--- a/modules/nodejs/src/main/js/compute.js
+++ b/modules/nodejs/src/main/js/compute.js
@@ -51,16 +51,6 @@ Compute.prototype.affinityCall = function(cacheName, key, runnable, callback) {
 }
 
 /**
- * @param {noValue} f Function
- * @returns {string} Encoding function
- */
-Compute.prototype._escape = function(f) {
-  var qs = require('querystring');
-
-  return qs.escape(f.toString());
-}
-
-/**
  * @this {Compute}
  * @param {MapFunction} map Map function
  * @param {ReduceFunction} reduce Reduce function
@@ -78,6 +68,16 @@ Compute.prototype.execute = function(map, reduce, arg, callback) {
 }
 
 /**
+ * @param {noValue} f Function
+ * @returns {string} Encoding function
+ */
+Compute.prototype._escape = function(f) {
+  var qs = require('querystring');
+
+  return qs.escape(f.toString());
+}
+
+/**
  * @name EmitFunction
  * @function
  * @param {function} func Remote job


[02/20] incubator-ignite git commit: #ignite-965: code style IgniteScriptProcessor.

Posted by sb...@apache.org.
#ignite-965: code style IgniteScriptProcessor.


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

Branch: refs/heads/ignite-965
Commit: 0a257f1809cc8e1dd3710558582478859325da9b
Parents: d481ea8
Author: ivasilinets <iv...@gridgain.com>
Authored: Tue Jun 23 18:27:42 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Tue Jun 23 18:27:42 2015 +0300

----------------------------------------------------------------------
 .../compute/IgniteComputeCommandHandler.java    | 14 +---
 .../IgniteComputeTaskCommandHandler.java        | 84 +++++++++-----------
 .../internal/processors/scripting/IgniteJS.java | 27 +++++++
 .../scripting/IgniteScriptProcessor.java        | 74 +++++++++--------
 .../internal/IgniteScriptManagerSelfTest.java   | 40 ----------
 5 files changed, 101 insertions(+), 138 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0a257f18/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/compute/IgniteComputeCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/compute/IgniteComputeCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/compute/IgniteComputeCommandHandler.java
index 345a898..acb4ab6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/compute/IgniteComputeCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/compute/IgniteComputeCommandHandler.java
@@ -69,12 +69,7 @@ public class IgniteComputeCommandHandler extends GridRestCommandHandlerAdapter {
                     private Ignite ignite;
 
                     @Override public void run() {
-                        try {
-                            ((IgniteKernal) ignite).context().scripting().runJS(req0.function());
-                        }
-                        catch (ScriptException e) {
-                            throw new IgniteException(e);
-                        }
+                        ((IgniteKernal) ignite).context().scripting().runJSFunction(req0.function());
                     }
                 });
 
@@ -86,12 +81,7 @@ public class IgniteComputeCommandHandler extends GridRestCommandHandlerAdapter {
                     private Ignite ignite;
 
                     @Override public Object call() {
-                        try {
-                            return ((IgniteKernal) ignite).context().scripting().runJS(req0.function());
-                        }
-                        catch (ScriptException e) {
-                            throw new IgniteException(e);
-                        }
+                        return ((IgniteKernal) ignite).context().scripting().runJSFunction(req0.function());
                     }
                 });
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0a257f18/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/compute/IgniteComputeTaskCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/compute/IgniteComputeTaskCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/compute/IgniteComputeTaskCommandHandler.java
index 7329a5b..8de073c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/compute/IgniteComputeTaskCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/compute/IgniteComputeTaskCommandHandler.java
@@ -99,61 +99,54 @@ public class IgniteComputeTaskCommandHandler extends GridRestCommandHandlerAdapt
 
             nodesIds = nodesIds.substring(0, nodesIds.length() - 1) + "]";
 
-            try {
-                String newMap = new String("function () {\n" +
-                    "   var res = [];\n" +
-                    "   var resCont = function(f, args, nodeId) {\n" +
-                    "       res.push([f.toString(), args, nodeId])\n" +
-                    "   }\n" +
-                    "   var locF = " + mapFunc + "; \n locF(" +
-                        nodesIds + ", " +
-                    "\"" + this.arg + "\"" +
-                    ", resCont.bind(null)" + ");\n" +
-                    "   return res;\n" +
-                    "}");
+            String newMap = new String("function () {\n" +
+                "   var res = [];\n" +
+                "   var resCont = function(f, args, nodeId) {\n" +
+                "       res.push([f.toString(), args, nodeId])\n" +
+                "   }\n" +
+                "   var locF = " + mapFunc + "; \n locF(" +
+                    nodesIds + ", " +
+                "\"" + this.arg + "\"" +
+                ", resCont.bind(null)" + ");\n" +
+                "   return res;\n" +
+                "}");
 
-                List mapRes = (List)ctx.scripting().runJS(newMap);
+            List mapRes = (List)ctx.scripting().runJSFunction(newMap);
 
-                for (Object arr : mapRes) {
-                    Object[] nodeTask = ((List)arr).toArray();
+            for (Object arr : mapRes) {
+                Object[] nodeTask = ((List)arr).toArray();
 
-                    final String func = (String)nodeTask[0];
+                final String func = (String)nodeTask[0];
 
-                    final List argv = (List) nodeTask[1];
+                final List argv = (List) nodeTask[1];
 
-                    String nodeIdStr = (String) nodeTask[2];
+                String nodeIdStr = (String) nodeTask[2];
 
-                    UUID nodeId = UUID.fromString(nodeIdStr);
+                UUID nodeId = UUID.fromString(nodeIdStr);
 
-                    ClusterNode node = ctx.grid().cluster().node(nodeId);
+                ClusterNode node = ctx.grid().cluster().node(nodeId);
 
-                    map.put(new ComputeJobAdapter() {
-                        /** Ignite. */
-                        @IgniteInstanceResource
-                        private Ignite ignite;
+                map.put(new ComputeJobAdapter() {
+                    /** Ignite. */
+                    @IgniteInstanceResource
+                    private Ignite ignite;
 
-                        @Override public Object execute() throws IgniteException {
-                            System.out.println("Compute job on node " + ignite.cluster().localNode().id());
-                            try {
-                                String[] argv1 = new String[argv.size()];
+                    @Override public Object execute() throws IgniteException {
+                        System.out.println("Compute job on node " + ignite.cluster().localNode().id());
+                        try {
+                            String[] argv1 = new String[argv.size()];
 
-                                for (int i = 0; i < argv1.length; ++i)
-                                    argv1[i] = "\"" + argv.get(i).toString() + "\"";
+                            for (int i = 0; i < argv1.length; ++i)
+                                argv1[i] = "\"" + argv.get(i).toString() + "\"";
 
-                                return ctx.scripting().runJS(func, argv1);
-                            }
-                            catch (Exception e) {
-                                throw new IgniteException(e);
-                            }
+                            return ctx.scripting().runJSFunction(func, argv1);
                         }
-                    }, node);
+                        catch (Exception e) {
+                            throw new IgniteException(e);
+                        }
+                    }
+                }, node);
 
-                }
-            }
-            catch (ScriptException e) {
-                throw new IgniteException(e);
-            }
-            finally {
             }
 
             return map;
@@ -166,12 +159,7 @@ public class IgniteComputeTaskCommandHandler extends GridRestCommandHandlerAdapt
             for (ComputeJobResult res : results)
                 data.add(res.getData());
 
-            try {
-                return ctx.scripting().runJS(reduceFunc, new String[] {data.toString()});
-            }
-            catch (ScriptException e) {
-                throw new IgniteException(e);
-            }
+            return ctx.scripting().runJSFunction(reduceFunc, new String[]{data.toString()});
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0a257f18/modules/core/src/main/java/org/apache/ignite/internal/processors/scripting/IgniteJS.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/scripting/IgniteJS.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/scripting/IgniteJS.java
new file mode 100644
index 0000000..6ffe1b1
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/scripting/IgniteJS.java
@@ -0,0 +1,27 @@
+/*
+ * 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.scripting;
+
+/**
+ * Ignite JS binding.
+ */
+public class IgniteJS {
+    public void hello() {
+        System.out.println("HELLO HAPPY WORLD!!!");
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0a257f18/modules/core/src/main/java/org/apache/ignite/internal/processors/scripting/IgniteScriptProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/scripting/IgniteScriptProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/scripting/IgniteScriptProcessor.java
index 99c206e..094373c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/scripting/IgniteScriptProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/scripting/IgniteScriptProcessor.java
@@ -20,20 +20,20 @@ package org.apache.ignite.internal.processors.scripting;
 import org.apache.ignite.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.*;
-import org.apache.ignite.internal.util.typedef.*;
 
 import javax.script.*;
-import java.util.*;
+
+import static javax.script.ScriptContext.*;
 
 /**
- * Ignite scripting manager.
+ * Ignite scripting processor.
  */
 public class IgniteScriptProcessor extends GridProcessorAdapter {
     /** Javascript engine name. */
     public static final String JAVA_SCRIPT_ENGINE_NAME = "JavaScript";
 
-    /** Script factory **/
-    private final ScriptEngineManager factory = new ScriptEngineManager();
+    /** Javascript engine. */
+    private ScriptEngine jsEngine;
 
     /**
      * @param ctx Kernal context.
@@ -42,16 +42,17 @@ public class IgniteScriptProcessor extends GridProcessorAdapter {
         super(ctx);
     }
 
-    /**
-     * @param engName Engine name.
-     * @param script Script.
-     * @throws ScriptException If script failed.
-     */
-    public Object run(String engName, String script) throws ScriptException {
-        if (!engName.equals(JAVA_SCRIPT_ENGINE_NAME))
-            throw new IgniteException("Engine is not supported. [engName=" + engName + "]");
+    /** {@inheritDoc} */
+    @Override public void start() throws IgniteCheckedException {
+        ScriptEngineManager factory = new ScriptEngineManager();
+
+        jsEngine = factory.getEngineByName(JAVA_SCRIPT_ENGINE_NAME);
 
-        return runJS(script);
+        Bindings bind = jsEngine.createBindings();
+
+        bind.put("ignite", new IgniteJS());
+
+        jsEngine.setBindings(bind, ENGINE_SCOPE);
     }
 
     /**
@@ -60,23 +61,13 @@ public class IgniteScriptProcessor extends GridProcessorAdapter {
      * @return Script result.
      * @throws ScriptException If script failed.
      */
-    public Object runJS(String script, String[] args) throws ScriptException {
-        ScriptEngine engine = factory.getEngineByName("JavaScript");
-
-        Bindings b = engine.createBindings();
-
-        b.put("ignite", new Ignite());
-
-        engine.setBindings(b, ScriptContext.ENGINE_SCOPE);
-
-        script = "(" + script + ")(" ;
-
-        for (int i = 0; i < args.length; ++i)
-            script += args[i] + (i < args.length - 1 ? "," : "");
-
-        script += ");";
-
-        return engine.eval(script);
+    public Object runJSFunction(String script, String[] args) throws IgniteException {
+        try {
+            return jsEngine.eval(callJsFunction(script, args));
+        }
+        catch (ScriptException e) {
+            throw new IgniteException("Cannot evaluate javascript function + " + script, e);
+        }
     }
 
     /**
@@ -84,16 +75,23 @@ public class IgniteScriptProcessor extends GridProcessorAdapter {
      * @return Script result.
      * @throws ScriptException If script failed.
      */
-    public Object runJS(String script) throws ScriptException {
-        return runJS(script, new String[]{""});
+    public Object runJSFunction(String script) throws IgniteException {
+        return runJSFunction(script, new String[]{""});
     }
 
     /**
-     * Ignite JS binding.
+     * @param script JS function script.
+     * @param args Arguments.
+     * @return Script that calls function.
      */
-    public static class Ignite {
-        public void hello() {
-            System.out.println("HELLO HAPPY WORLD!!!");
-        }
+    private String callJsFunction(String script, String[] args) {
+        String callFuncScript = "(" + script + ")(";
+
+        for (int i = 0; i < args.length; ++i)
+            callFuncScript += args[i] + (i < args.length - 1 ? "," : "");
+
+        callFuncScript += ");";
+
+        return callFuncScript;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0a257f18/modules/nodejs/src/test/java/org/apache/ignite/internal/IgniteScriptManagerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/nodejs/src/test/java/org/apache/ignite/internal/IgniteScriptManagerSelfTest.java b/modules/nodejs/src/test/java/org/apache/ignite/internal/IgniteScriptManagerSelfTest.java
deleted file mode 100644
index e61fd26..0000000
--- a/modules/nodejs/src/test/java/org/apache/ignite/internal/IgniteScriptManagerSelfTest.java
+++ /dev/null
@@ -1,40 +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;
-
-import org.apache.ignite.internal.processors.scripting.*;
-import org.apache.ignite.testframework.junits.common.*;
-
-/**
- * Test {@link IgniteScriptProcessor}
- */
-public class IgniteScriptManagerSelfTest extends GridCommonAbstractTest {
-    /**
-     * @throws Exception If failed.
-     */
-    public void testRunScript() throws Exception {
-        IgniteScriptProcessor mng = startGrid(0).context().scripting();
-
-        System.out.println("Result = " + mng.runJS("5 + 5;"));
-        System.out.println("Result = " + mng.runJS("function () {return (5+5)}"));
-
-        System.out.println("Result = " + mng.runJS("function sum() {return (5+5)}; sum(); "));
-
-        stopAllGrids();
-    }
-}


[13/20] incubator-ignite git commit: #ignite-965: code style cache.js

Posted by sb...@apache.org.
#ignite-965: code style cache.js


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

Branch: refs/heads/ignite-965
Commit: 905a3e6aa80c0510282ff1b342df6c8d0a2e505a
Parents: ac5434a
Author: ivasilinets <iv...@gridgain.com>
Authored: Tue Jun 23 20:12:57 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Tue Jun 23 20:12:57 2015 +0300

----------------------------------------------------------------------
 modules/nodejs/src/main/js/cache.js  | 9 ++-------
 modules/nodejs/src/main/js/server.js | 5 +++++
 2 files changed, 7 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/905a3e6a/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 8a62be9..78dd1d8 100644
--- a/modules/nodejs/src/main/js/cache.js
+++ b/modules/nodejs/src/main/js/cache.js
@@ -43,12 +43,6 @@ Cache.prototype.get = function(key, callback) {
 };
 
 /**
- * Callback for cache put
- * @callback noValue
- * @param {string} error Error
- */
-
-/**
  * Put cache value
  *
  * @this {Cache}
@@ -91,7 +85,7 @@ Cache.prototype.removeAll = function(keys, callback) {
  * Put keys to cache
  *
  * @this {Cache}
- * @param {Object.<string, string>} collection of entries to put in the cache
+ * @param {Object.<string, string>} map collection of entries to put in the cache
  * @param {noValue} callback Called on finish
  */
 Cache.prototype.putAll = function(map, callback) {
@@ -113,6 +107,7 @@ Cache.prototype.putAll = function(map, callback) {
 
 /**
  * Callback for cache get
+ *
  * @callback Cache~onGetAll
  * @param {string} error Error
  * @param {string[]} results Result values

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/905a3e6a/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 c1b52c0..910b7c3 100644
--- a/modules/nodejs/src/main/js/server.js
+++ b/modules/nodejs/src/main/js/server.js
@@ -49,6 +49,11 @@ Server.prototype.host = function() {
  */
 
 /**
+ * @callback noValue
+ * @param {string} error Error
+ */
+
+/**
  * Run http request
  *
  * @this {Server}


[16/20] incubator-ignite git commit: #ignite-965: code style ignite.js

Posted by sb...@apache.org.
#ignite-965: code style ignite.js


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

Branch: refs/heads/ignite-965
Commit: 294531bf18cf2d6553c11de1e212f7ab93f14b4f
Parents: 4bb2004
Author: ivasilinets <iv...@gridgain.com>
Authored: Tue Jun 23 20:22:44 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Tue Jun 23 20:22:44 2015 +0300

----------------------------------------------------------------------
 modules/nodejs/src/main/js/ignite.js | 9 +++++----
 1 file changed, 5 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/294531bf/modules/nodejs/src/main/js/ignite.js
----------------------------------------------------------------------
diff --git a/modules/nodejs/src/main/js/ignite.js b/modules/nodejs/src/main/js/ignite.js
index 1db3be8..dca292e 100644
--- a/modules/nodejs/src/main/js/ignite.js
+++ b/modules/nodejs/src/main/js/ignite.js
@@ -15,6 +15,9 @@
  * limitations under the License.
  */
 
+var Cache = require("./cache").Cache;
+var Compute = require("./compute").Compute
+
 /**
  * Create an instance of Ignite
  *
@@ -27,6 +30,7 @@ function Ignite(server) {
 }
 
 /**
+ * @this {Ignite}
  * @returns {Server} Server
  */
 Ignite.prototype.server = function() {
@@ -41,8 +45,6 @@ Ignite.prototype.server = function() {
  * @returns {Cache} Cache
  */
 Ignite.prototype.cache = function(cacheName) {
-  var Cache = require("./cache").Cache;
-
   return new Cache(this._server, cacheName);
 }
 
@@ -53,8 +55,7 @@ Ignite.prototype.cache = function(cacheName) {
  * @returns {Compute} Compute
  */
 Ignite.prototype.compute = function() {
-  var Compute = require("./compute").Compute
-
   return new Compute(this._server);
 }
+
 exports.Ignite = Ignite;


[04/20] incubator-ignite git commit: #ignite-965: add map reduce Javadoc

Posted by sb...@apache.org.
#ignite-965: add map reduce Javadoc


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

Branch: refs/heads/ignite-965
Commit: 30ec9f75f7198beea1465ea2c50391f29b2f5375
Parents: d24ba84
Author: ivasilinets <iv...@gridgain.com>
Authored: Tue Jun 23 19:05:10 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Tue Jun 23 19:05:10 2015 +0300

----------------------------------------------------------------------
 .../IgniteComputeTaskCommandHandler.java        |  1 -
 modules/nodejs/src/main/js/cache.js             | 19 ++++-------
 modules/nodejs/src/main/js/compute.js           | 33 ++++++++++++++++++++
 modules/nodejs/src/main/js/server.js            |  6 ++--
 .../testsuites/IgniteNodeJsTestSuite.java       |  1 +
 5 files changed, 43 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/30ec9f75/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/compute/IgniteComputeTaskCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/compute/IgniteComputeTaskCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/compute/IgniteComputeTaskCommandHandler.java
index 8de073c..33f4893 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/compute/IgniteComputeTaskCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/compute/IgniteComputeTaskCommandHandler.java
@@ -29,7 +29,6 @@ import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.resources.*;
 import org.jetbrains.annotations.*;
 
-import javax.script.*;
 import java.util.*;
 
 import static org.apache.ignite.internal.processors.rest.GridRestCommand.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/30ec9f75/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 ee4452e..8a62be9 100644
--- a/modules/nodejs/src/main/js/cache.js
+++ b/modules/nodejs/src/main/js/cache.js
@@ -32,18 +32,11 @@ function Cache(server, cacheName) {
 }
 
 /**
- * Callback for cache get
- * @callback Cache~onGet
- * @param {string} error Error
- * @param {string} result Result value
- */
-
-/**
  * Get cache value
  *
  * @this {Cache}
  * @param {string} key Key
- * @param {Cache~onGet} callback Called on finish
+ * @param {onGet} callback Called on finish
  */
 Cache.prototype.get = function(key, callback) {
   this._server.runCommand("get", [this._cacheNameParam, Server.pair("key", key)], callback);
@@ -51,7 +44,7 @@ Cache.prototype.get = function(key, callback) {
 
 /**
  * Callback for cache put
- * @callback Cache~noValue
+ * @callback noValue
  * @param {string} error Error
  */
 
@@ -61,7 +54,7 @@ Cache.prototype.get = function(key, callback) {
  * @this {Cache}
  * @param {string} key Key
  * @param {string} value Value
- * @param {Cache~noValue} callback Called on finish
+ * @param {noValue} callback Called on finish
  */
 Cache.prototype.put = function(key, value, callback) {
   this._server.runCommand("put", [this._cacheNameParam, Server.pair("key", key), Server.pair("val", value)],
@@ -73,7 +66,7 @@ Cache.prototype.put = function(key, value, callback) {
  *
  * @this {Cache}
  * @param {string} key Key
- * @param {Cache~noValue} callback Called on finish
+ * @param {noValue} callback Called on finish
  */
 Cache.prototype.remove = function(key, callback) {
   this._server.runCommand("rmv", [this._cacheNameParam, Server.pair("key", key)], callback);
@@ -84,7 +77,7 @@ Cache.prototype.remove = function(key, callback) {
  *
  * @this {Cache}
  * @param {string[]} keys Keys to remove
- * @param {Cache~noValue} callback Called on finish
+ * @param {noValue} callback Called on finish
  */
 Cache.prototype.removeAll = function(keys, callback) {
   var params = [this._cacheNameParam];
@@ -99,7 +92,7 @@ Cache.prototype.removeAll = function(keys, callback) {
  *
  * @this {Cache}
  * @param {Object.<string, string>} collection of entries to put in the cache
- * @param {Cache~noValue} callback Called on finish
+ * @param {noValue} callback Called on finish
  */
 Cache.prototype.putAll = function(map, callback) {
   var keys = Object.keys(map);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/30ec9f75/modules/nodejs/src/main/js/compute.js
----------------------------------------------------------------------
diff --git a/modules/nodejs/src/main/js/compute.js b/modules/nodejs/src/main/js/compute.js
index 7a5f808..7f45f5b 100644
--- a/modules/nodejs/src/main/js/compute.js
+++ b/modules/nodejs/src/main/js/compute.js
@@ -81,4 +81,37 @@ Compute.prototype.execute = function(map, reduce, arg, callback) {
     this._server.runCommand("exectask", params, callback);
 }
 
+/**
+ * @name EmitFunction
+ * @function
+ * @param {function} func Remote job
+ * @param {string[]} args Arguments for remote job
+ * @param {string} node Node Id to call job on.
+ */
+
+/**
+ * @name MapFunction
+ * @function
+ * @param {string[]} nodes Nodes Id
+ * @param {string} arg Argument
+ * @param {EmitFunction} emit Emit function to call for adding to result
+ */
+
+/**
+ * @name ReduceFunction
+ * @function
+ * @param {string[]} results Results of executing jobs after mapping
+ * @returns {string} Result
+ */
+
+/**
+ * @this{Compute}
+ * @param {MapFunction} map Map function
+ * @param {ReduceFunction} reduce Reduce function
+ * @param {string} arg Argument
+ * @param {onGet} callback Callback
+ */
+Compute.prototype._mapWrapper = function(map) {
+
+}
 exports.Compute = Compute

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/30ec9f75/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 d4660ae..c1b52c0 100644
--- a/modules/nodejs/src/main/js/server.js
+++ b/modules/nodejs/src/main/js/server.js
@@ -43,7 +43,7 @@ Server.prototype.host = function() {
 /**
  * Callback for Server runCommand
  *
- * @callback Server~onRunCommand
+ * @callback onGet
  * @param {string} error Error
  * @param {string} result Result value
  */
@@ -54,7 +54,7 @@ Server.prototype.host = function() {
  * @this {Server}
  * @param {string} cmdName command name.
  * @param params Parameters for command.
- * @param {Server~onRunCommand} Called on finish
+ * @param {onGet} Called on finish
  */
 Server.prototype.runCommand = function(cmdName, params, callback) {
   var paramsString = "";
@@ -126,7 +126,7 @@ Server.prototype.runCommand = function(cmdName, params, callback) {
  * Check the connection with server node.
  *
  * @this {Server}
- * @param {Server~onRunCommand} callback Called on finish
+ * @param {onGet} callback Called on finish
  */
 Server.prototype.checkConnection = function(callback) {
   this.runCommand("version", [], callback);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/30ec9f75/modules/nodejs/src/test/java/org/apache/ignite/testsuites/IgniteNodeJsTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/nodejs/src/test/java/org/apache/ignite/testsuites/IgniteNodeJsTestSuite.java b/modules/nodejs/src/test/java/org/apache/ignite/testsuites/IgniteNodeJsTestSuite.java
index 9d5daee..eb69d7f 100644
--- a/modules/nodejs/src/test/java/org/apache/ignite/testsuites/IgniteNodeJsTestSuite.java
+++ b/modules/nodejs/src/test/java/org/apache/ignite/testsuites/IgniteNodeJsTestSuite.java
@@ -34,6 +34,7 @@ public class IgniteNodeJsTestSuite extends TestSuite {
         suite.addTest(new TestSuite(NodeJsIgnitionSelfTest.class));
         suite.addTest(new TestSuite(NodeJsCacheApiSelfTest.class));
         suite.addTest(new TestSuite(NodeJsSecretKeySelfTest.class));
+        suite.addTest(new TestSuite(NodeJsComputeSelfTest.class));
 
         return suite;
     }


[20/20] incubator-ignite git commit: #ignite-965: code style test-compute.js

Posted by sb...@apache.org.
#ignite-965: code style test-compute.js


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

Branch: refs/heads/ignite-965
Commit: 47276a81e715a5525a66a875d37e13892bf13216
Parents: d7e5465
Author: ivasilinets <iv...@gridgain.com>
Authored: Tue Jun 23 20:47:03 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Tue Jun 23 20:47:03 2015 +0300

----------------------------------------------------------------------
 .../internal/processors/scripting/IgniteJS.java |  8 ++-
 modules/nodejs/src/test/js/test-compute.js      | 66 +++++++++-----------
 2 files changed, 34 insertions(+), 40 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/47276a81/modules/core/src/main/java/org/apache/ignite/internal/processors/scripting/IgniteJS.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/scripting/IgniteJS.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/scripting/IgniteJS.java
index 6ffe1b1..dd7624c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/scripting/IgniteJS.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/scripting/IgniteJS.java
@@ -21,7 +21,11 @@ package org.apache.ignite.internal.processors.scripting;
  * Ignite JS binding.
  */
 public class IgniteJS {
-    public void hello() {
-        System.out.println("HELLO HAPPY WORLD!!!");
+    public String hello() {
+        String str = "HELLO HAPPY WORLD!!!";
+
+        System.out.println(str);
+
+        return str;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/47276a81/modules/nodejs/src/test/js/test-compute.js
----------------------------------------------------------------------
diff --git a/modules/nodejs/src/test/js/test-compute.js b/modules/nodejs/src/test/js/test-compute.js
index c444380..2267662 100644
--- a/modules/nodejs/src/test/js/test-compute.js
+++ b/modules/nodejs/src/test/js/test-compute.js
@@ -20,29 +20,30 @@ var TestUtils = require("./test-utils").TestUtils;
 var assert = require("assert");
 
 testComputeAffinityRun = function() {
-  TestUtils.startIgniteNode(onStart.bind(null, onPut));
+  TestUtils.startIgniteNode(onStart.bind(null, computeAffinityRun));
 }
 
 testComputeAffinityCall = function() {
-  TestUtils.startIgniteNode(onStart.bind(null, onPut1));
+  TestUtils.startIgniteNode(onStart.bind(null, computeAffinityCall));
 }
 
 testComputeExecute = function() {
-  TestUtils.startIgniteNode(onStart1);
+  TestUtils.startIgniteNode(computeExecute);
 }
 
-function onStart(locOnPut, error, ignite) {
+function onStart(onPut, error, ignite) {
   var cache = ignite.cache("mycache");
 
   var params = {}
 
-  for (var i = 900; i < 1000; ++i)
+  for (var i = 900; i < 1000; ++i) {
     params["key" + i] = "val" + i;
+  }
 
-  cache.putAll(params, locOnPut.bind(null, ignite))
+  cache.putAll(params, onPut.bind(null, ignite))
 }
 
-function onPut(ignite, error) {
+function computeAffinityRun(ignite, error) {
   var comp = ignite.compute();
 
   var f = function () {
@@ -51,40 +52,34 @@ function onPut(ignite, error) {
     ignite.hello();
   }
 
-  comp.affinityRun("mycache", "key999", f, onError.bind(null));
-}
-
-function onError(error) {
-  console.log("Error "  + error);
+  function onEnd(error) {
+    assert(error == null);
 
-  assert(error == null);
+    TestUtils.testDone();
+  }
 
-  TestUtils.testDone();
+  comp.affinityRun("mycache", "key999", f, onEnd.bind(null));
 }
 
-function onPut1(ignite, error) {
+function computeAffinityCall(ignite, error) {
   var comp = ignite.compute();
 
   var f = function () {
-    println("Hello world!");
-
-    ignite.hello();
+    return ignite.hello();
   }
 
-  comp.affinityCall("mycache", "key999", f, onError1.bind(null));
-}
+  function onEnd(err, res) {
+    assert(err == null);
 
-function onError1(error, res) {
-  console.log("Error "  + error);
+    assert(res.indexOf("HAPPY") !== -1, "Incorrect result message. [mes=" + res + "].");
 
-  assert(error == null);
-
-  console.log("!!!!!!!!RES = " + res);
+    TestUtils.testDone();
+  }
 
-  TestUtils.testDone();
+  comp.affinityCall("mycache", "key999", f, onEnd.bind(null));
 }
 
-function onStart1(error, ignite) {
+function computeExecute(error, ignite) {
   var map = function(nodes, arg, emit) {
     var words = arg.split(" ");
 
@@ -108,17 +103,12 @@ function onStart1(error, ignite) {
     return sum;
   };
 
-  ignite.compute().execute(map, reduce, "Hi Alice", onComputeResult);
-}
-
-function onComputeResult(error, res) {
-  console.log("Error "  + error);
+  var callback = function(err, res) {
+    assert(err == null, "Get error on compute task. [err=" + err + "].");
+    assert(res === 7, "Result is not correct. [expected=7, value=" + res + "].");
 
-  assert(error == null);
-
-  console.log("!!!!!!!!EXECUTE TASK RESULT = " + res);
-
-  assert(res === 7, "Result is not correct. [expected=7, value=" + res + "].")
+    TestUtils.testDone();
+  }
 
-  TestUtils.testDone();
+  ignite.compute().execute(map, reduce, "Hi Alice", callback);
 }


[12/20] incubator-ignite git commit: #ignite-965: code style node js java tests.

Posted by sb...@apache.org.
#ignite-965: code style node js java tests.


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

Branch: refs/heads/ignite-965
Commit: ac5434a44421e057d3642b77f5be43ea711e2ee0
Parents: 53ed5c0
Author: ivasilinets <iv...@gridgain.com>
Authored: Tue Jun 23 20:04:09 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Tue Jun 23 20:04:09 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/NodeJsAbstractTest.java | 17 ++++++-----------
 .../ignite/internal/NodeJsCacheApiSelfTest.java    |  4 +++-
 .../ignite/internal/NodeJsComputeSelfTest.java     |  4 +++-
 .../ignite/internal/NodeJsIgnitionSelfTest.java    |  4 +++-
 4 files changed, 15 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ac5434a4/modules/nodejs/src/test/java/org/apache/ignite/internal/NodeJsAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/nodejs/src/test/java/org/apache/ignite/internal/NodeJsAbstractTest.java b/modules/nodejs/src/test/java/org/apache/ignite/internal/NodeJsAbstractTest.java
index 5b7af7b..79ae908 100644
--- a/modules/nodejs/src/test/java/org/apache/ignite/internal/NodeJsAbstractTest.java
+++ b/modules/nodejs/src/test/java/org/apache/ignite/internal/NodeJsAbstractTest.java
@@ -22,9 +22,9 @@ import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
-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.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
 import org.apache.ignite.testframework.junits.common.*;
 
 import java.util.*;
@@ -48,7 +48,7 @@ public class NodeJsAbstractTest extends GridCommonAbstractTest {
     /** Node JS file with tests. */
     private String fileName;
 
-    /** */
+    /** Ip finder. */
     private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
 
     /**
@@ -61,6 +61,7 @@ public class NodeJsAbstractTest extends GridCommonAbstractTest {
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
+
         cfg.setCacheConfiguration(cacheConfiguration());
 
         ConnectorConfiguration conCfg = new ConnectorConfiguration();
@@ -118,11 +119,8 @@ public class NodeJsAbstractTest extends GridCommonAbstractTest {
         List<String> cmd = new ArrayList<>();
 
         cmd.add("node");
-
         cmd.add(getNodeJsTestDir() + "test-runner.js");
-
         cmd.add(fileName);
-
         cmd.add(functionName);
 
         Map<String, String> env = new HashMap<>();
@@ -130,10 +128,7 @@ public class NodeJsAbstractTest extends GridCommonAbstractTest {
         env.put("IGNITE_HOME", IgniteUtils.getIgniteHome());
 
         try {
-            proc = GridJavaProcess.exec(
-                cmd,
-                env,
-                log,
+            proc = GridJavaProcess.exec(cmd, env, log,
                 new CI1<String>() {
                     @Override public void apply(String s) {
                         info("Node js: " + s);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ac5434a4/modules/nodejs/src/test/java/org/apache/ignite/internal/NodeJsCacheApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/nodejs/src/test/java/org/apache/ignite/internal/NodeJsCacheApiSelfTest.java b/modules/nodejs/src/test/java/org/apache/ignite/internal/NodeJsCacheApiSelfTest.java
index 6c02ede..f17ca60 100644
--- a/modules/nodejs/src/test/java/org/apache/ignite/internal/NodeJsCacheApiSelfTest.java
+++ b/modules/nodejs/src/test/java/org/apache/ignite/internal/NodeJsCacheApiSelfTest.java
@@ -21,7 +21,9 @@ package org.apache.ignite.internal;
  * Test node js client put/get.
  */
 public class NodeJsCacheApiSelfTest extends NodeJsAbstractTest {
-    /** Constructor. */
+    /**
+     * Constructor.
+     */
     public NodeJsCacheApiSelfTest() {
         super("test-cache-api.js");
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ac5434a4/modules/nodejs/src/test/java/org/apache/ignite/internal/NodeJsComputeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/nodejs/src/test/java/org/apache/ignite/internal/NodeJsComputeSelfTest.java b/modules/nodejs/src/test/java/org/apache/ignite/internal/NodeJsComputeSelfTest.java
index e6ec550..5c4635d 100644
--- a/modules/nodejs/src/test/java/org/apache/ignite/internal/NodeJsComputeSelfTest.java
+++ b/modules/nodejs/src/test/java/org/apache/ignite/internal/NodeJsComputeSelfTest.java
@@ -21,7 +21,9 @@ package org.apache.ignite.internal;
  * Test compute node js.
  */
 public class NodeJsComputeSelfTest extends NodeJsAbstractTest {
-    /** Constructor. */
+    /**
+     * Constructor.
+     */
     public NodeJsComputeSelfTest() {
         super("test-compute.js");
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ac5434a4/modules/nodejs/src/test/java/org/apache/ignite/internal/NodeJsIgnitionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/nodejs/src/test/java/org/apache/ignite/internal/NodeJsIgnitionSelfTest.java b/modules/nodejs/src/test/java/org/apache/ignite/internal/NodeJsIgnitionSelfTest.java
index 08973d5..205e467 100644
--- a/modules/nodejs/src/test/java/org/apache/ignite/internal/NodeJsIgnitionSelfTest.java
+++ b/modules/nodejs/src/test/java/org/apache/ignite/internal/NodeJsIgnitionSelfTest.java
@@ -21,7 +21,9 @@ package org.apache.ignite.internal;
  * Test node js client.
  */
 public class NodeJsIgnitionSelfTest extends NodeJsAbstractTest {
-    /** Constructor. */
+    /**
+     * Constructor.
+     */
     public NodeJsIgnitionSelfTest() {
         super("test-ignition.js");
     }