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/19 18:22:05 UTC

incubator-ignite git commit: #ignite-965: add affinity run to nodejs - run locally hello world.

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-965 988016dce -> e77e76acc


#ignite-965: add affinity run to nodejs - run locally hello world.


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

Branch: refs/heads/ignite-965
Commit: e77e76acc4c1cc97c61a079a4cff475b79abab1a
Parents: 988016d
Author: ivasilinets <iv...@gridgain.com>
Authored: Fri Jun 19 19:21:55 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Fri Jun 19 19:21:55 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/GridKernalContext.java      |  8 +++
 .../ignite/internal/GridKernalContextImpl.java  | 12 ++++
 .../apache/ignite/internal/IgniteKernal.java    |  2 +
 .../processors/rest/GridRestCommand.java        |  5 +-
 .../processors/rest/GridRestProcessor.java      |  2 +
 .../compute/IgniteComputeCommandHandler.java    | 69 ++++++++++++++++++++
 .../rest/request/RestComputeRequest.java        | 40 ++++++++++++
 .../scripting/IgniteScriptProcessor.java        | 66 +++++++++++++++++++
 modules/nodejs/src/main/js/cache.js             |  1 +
 modules/nodejs/src/main/js/compute.js           | 45 +++++++++++++
 modules/nodejs/src/main/js/ignite.js            | 11 ++++
 modules/nodejs/src/main/js/server.js            |  1 +
 .../internal/IgniteScriptManagerSelfTest.java   | 40 ++++++++++++
 .../ignite/internal/NodeJsComputeSelfTest.java  | 50 ++++++++++++++
 modules/nodejs/src/test/js/test-compute.js      | 50 ++++++++++++++
 .../http/jetty/GridJettyRestHandler.java        | 10 +++
 16 files changed, 411 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e77e76ac/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
index d6542f3..16615fa 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
@@ -49,6 +49,7 @@ import org.apache.ignite.internal.processors.query.*;
 import org.apache.ignite.internal.processors.resource.*;
 import org.apache.ignite.internal.processors.rest.*;
 import org.apache.ignite.internal.processors.schedule.*;
+import org.apache.ignite.internal.processors.scripting.*;
 import org.apache.ignite.internal.processors.security.*;
 import org.apache.ignite.internal.processors.segmentation.*;
 import org.apache.ignite.internal.processors.service.*;
@@ -235,6 +236,13 @@ public interface GridKernalContext extends Iterable<GridComponent> {
     public GridRestProcessor rest();
 
     /**
+     * Gets Scripting processor.
+     *
+     * @return Scripting processor.
+     */
+    public IgniteScriptProcessor scripting();
+
+    /**
      * Gets segmentation processor.
      *
      * @return Segmentation processor.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e77e76ac/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
index 7b07593..d2e9ca1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
@@ -49,6 +49,7 @@ import org.apache.ignite.internal.processors.query.*;
 import org.apache.ignite.internal.processors.resource.*;
 import org.apache.ignite.internal.processors.rest.*;
 import org.apache.ignite.internal.processors.schedule.*;
+import org.apache.ignite.internal.processors.scripting.*;
 import org.apache.ignite.internal.processors.security.*;
 import org.apache.ignite.internal.processors.segmentation.*;
 import org.apache.ignite.internal.processors.service.*;
@@ -198,6 +199,10 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
 
     /** */
     @GridToStringInclude
+    private IgniteScriptProcessor scriptProc;
+
+    /** */
+    @GridToStringInclude
     private DataStreamProcessor dataLdrProc;
 
     /** */
@@ -477,6 +482,8 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
             dataStructuresProc = (DataStructuresProcessor)comp;
         else if (comp instanceof ClusterProcessor)
             cluster = (ClusterProcessor)comp;
+        else if (comp instanceof IgniteScriptProcessor)
+            scriptProc = (IgniteScriptProcessor) comp;
         else if (!(comp instanceof DiscoveryNodeValidationProcessor))
             assert (comp instanceof GridPluginComponent) : "Unknown manager class: " + comp.getClass();
 
@@ -657,6 +664,11 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
     }
 
     /** {@inheritDoc} */
+    @Override public IgniteScriptProcessor scripting() {
+        return scriptProc;
+    }
+
+    /** {@inheritDoc} */
     @Override public GridSegmentationProcessor segmentation() {
         return segProc;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e77e76ac/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index e19d3d3..852616b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -53,6 +53,7 @@ import org.apache.ignite.internal.processors.port.*;
 import org.apache.ignite.internal.processors.query.*;
 import org.apache.ignite.internal.processors.resource.*;
 import org.apache.ignite.internal.processors.rest.*;
+import org.apache.ignite.internal.processors.scripting.*;
 import org.apache.ignite.internal.processors.security.*;
 import org.apache.ignite.internal.processors.segmentation.*;
 import org.apache.ignite.internal.processors.service.*;
@@ -776,6 +777,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
             startProcessor(new GridTaskProcessor(ctx));
             startProcessor((GridProcessor)SCHEDULE.createOptional(ctx));
             startProcessor(new GridRestProcessor(ctx));
+            startProcessor(new IgniteScriptProcessor(ctx));
             startProcessor(new DataStreamProcessor(ctx));
             startProcessor((GridProcessor)IGFS.create(ctx, F.isEmpty(cfg.getFileSystemConfiguration())));
             startProcessor(new GridContinuousProcessor(ctx));

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e77e76ac/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 62732f0..a539180 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
@@ -94,7 +94,10 @@ public enum GridRestCommand {
     NOOP("noop"),
 
     /** Quit. */
-    QUIT("quit");
+    QUIT("quit"),
+
+    /** Affinity run. */
+    AFFINITY_RUN("affrun");
 
     /** Enum values. */
     private static final GridRestCommand[] VALS = values();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e77e76ac/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java
index 52ca610..3377dcd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java
@@ -24,6 +24,7 @@ import org.apache.ignite.internal.processors.*;
 import org.apache.ignite.internal.processors.rest.client.message.*;
 import org.apache.ignite.internal.processors.rest.handlers.*;
 import org.apache.ignite.internal.processors.rest.handlers.cache.*;
+import org.apache.ignite.internal.processors.rest.handlers.compute.*;
 import org.apache.ignite.internal.processors.rest.handlers.datastructures.*;
 import org.apache.ignite.internal.processors.rest.handlers.task.*;
 import org.apache.ignite.internal.processors.rest.handlers.top.*;
@@ -252,6 +253,7 @@ public class GridRestProcessor extends GridProcessorAdapter {
             addHandler(new GridTopologyCommandHandler(ctx));
             addHandler(new GridVersionCommandHandler(ctx));
             addHandler(new DataStructuresCommandHandler(ctx));
+            addHandler(new IgniteComputeCommandHandler(ctx));
 
             // Start protocols.
             startTcpProtocol();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e77e76ac/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
new file mode 100644
index 0000000..b48e2d4
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/compute/IgniteComputeCommandHandler.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.rest.handlers.compute;
+
+import org.apache.ignite.*;
+import org.apache.ignite.internal.*;
+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.internal.*;
+
+import javax.script.*;
+import java.util.*;
+
+import static org.apache.ignite.internal.processors.rest.GridRestCommand.*;
+
+/**
+ * Compute command handler.
+ */
+public class IgniteComputeCommandHandler extends GridRestCommandHandlerAdapter {
+    /** Supported commands. */
+    private static final Collection<GridRestCommand> SUPPORTED_COMMANDS = U.sealList(AFFINITY_RUN);
+
+    /**
+     * @param ctx Context.
+     */
+    public IgniteComputeCommandHandler(GridKernalContext ctx) {
+        super(ctx);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<GridRestCommand> supportedCommands() {
+        return SUPPORTED_COMMANDS;
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteInternalFuture<GridRestResponse> handleAsync(GridRestRequest req) {
+        assert req != null;
+
+        assert req instanceof RestComputeRequest : "Invalid type of compute request.";
+
+        assert SUPPORTED_COMMANDS.contains(req.command());
+
+        try {
+            ctx.scripting().runJS(((RestComputeRequest) req).function());
+        }
+        catch (ScriptException e) {
+            throw new IgniteException(e);
+        }
+
+        return new GridFinishedFuture<>(new GridRestResponse("AFFINITY RUN " + req));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e77e76ac/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/RestComputeRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/RestComputeRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/RestComputeRequest.java
new file mode 100644
index 0000000..b046d78
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/RestComputeRequest.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.rest.request;
+
+/**
+ * Compute request.
+ */
+public class RestComputeRequest extends GridRestRequest {
+    /** Java script function. */
+    private String func;
+
+    /**
+     * @return Java script function.
+     */
+    public String function() {
+        return func;
+    }
+
+    /**
+     * @param func Java script function.
+     */
+    public void function(String func) {
+        this.func = func;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e77e76ac/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
new file mode 100644
index 0000000..bf0d063
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/scripting/IgniteScriptProcessor.java
@@ -0,0 +1,66 @@
+/*
+ * 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;
+
+import org.apache.ignite.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.processors.*;
+
+import javax.script.*;
+
+/**
+ * Ignite scripting manager.
+ */
+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();
+
+    /**
+     * @param ctx Kernal context.
+     */
+    public IgniteScriptProcessor(GridKernalContext ctx) {
+        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 + "]");
+
+        return runJS(script);
+    }
+
+    /**
+     * @param script Script.
+     * @throws ScriptException If script failed.
+     */
+    public Object runJS(String script) throws ScriptException {
+        ScriptEngine engine = factory.getEngineByName("JavaScript");
+
+        script = "(" + script + ")();";
+
+        return engine.eval(script);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e77e76ac/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 bf4089b..ee4452e 100644
--- a/modules/nodejs/src/main/js/cache.js
+++ b/modules/nodejs/src/main/js/cache.js
@@ -156,4 +156,5 @@ Cache.concatParams = function(pref, keys) {
 
   return temp;
 }
+
 exports.Cache = Cache
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e77e76ac/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
new file mode 100644
index 0000000..d68b6ef
--- /dev/null
+++ b/modules/nodejs/src/main/js/compute.js
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+var Server = require("./server").Server;
+
+/**
+ * @constructor
+ * @this {Compute}
+ * @param {Server} server Server class
+ */
+function Compute(server) {
+  this._server = server;
+}
+
+/**
+ * Callback for affinityRun
+ * @callback Compute~runnable
+ */
+
+/**
+ * @this {Compute}
+ * @param {Compute~runnable} runnable Function without parameters
+ */
+Compute.prototype.affinityRun = function(runnable, callback) {
+  var f = runnable.toString();
+  var qs = require('querystring');
+  f = qs.escape(f);
+  this._server.runCommand("affrun", [Server.pair("func", f)], callback);
+}
+
+exports.Compute = Compute;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e77e76ac/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 2f3ff44..1db3be8 100644
--- a/modules/nodejs/src/main/js/ignite.js
+++ b/modules/nodejs/src/main/js/ignite.js
@@ -46,4 +46,15 @@ Ignite.prototype.cache = function(cacheName) {
   return new Cache(this._server, cacheName);
 }
 
+/**
+ * Get an instance of compute
+ *
+ * @this {Ignite}
+ * @returns {Compute} Compute
+ */
+Ignite.prototype.compute = function() {
+  var Compute = require("./compute").Compute
+
+  return new Compute(this._server);
+}
 exports.Ignite = Ignite;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e77e76ac/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 b1a3190..d4660ae 100644
--- a/modules/nodejs/src/main/js/server.js
+++ b/modules/nodejs/src/main/js/server.js
@@ -112,6 +112,7 @@ 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"));

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e77e76ac/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
new file mode 100644
index 0000000..e61fd26
--- /dev/null
+++ b/modules/nodejs/src/test/java/org/apache/ignite/internal/IgniteScriptManagerSelfTest.java
@@ -0,0 +1,40 @@
+/*
+ * 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();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e77e76ac/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
new file mode 100644
index 0000000..500e6a2
--- /dev/null
+++ b/modules/nodejs/src/test/java/org/apache/ignite/internal/NodeJsComputeSelfTest.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal;
+
+/**
+ * Test compute node js.
+ */
+public class NodeJsComputeSelfTest extends NodeJsAbstractTest {
+    /** Constructor. */
+    public NodeJsComputeSelfTest() {
+        super("test-compute.js");
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        startGrid(0);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        grid(0).cache(NodeJsAbstractTest.CACHE_NAME).removeAll();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCompute() throws Exception {
+        runJsScript("testCompute");
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e77e76ac/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
new file mode 100644
index 0000000..30ff686
--- /dev/null
+++ b/modules/nodejs/src/test/js/test-compute.js
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+var TestUtils = require("./test-utils").TestUtils;
+
+var Apache = require(TestUtils.scriptPath());
+var Cache = Apache.Cache;
+var Server = Apache.Server;
+
+var assert = require("assert");
+
+testCompute = function() {
+  TestUtils.startIgniteNode(onStart.bind(null));
+}
+
+function onStart(error, ignite) {
+  var comp = ignite.compute();
+
+  var f = function () {
+    print("Hello world!");
+  }
+
+  comp.affinityRun(f, onError.bind(null));
+}
+
+function onError(error, res) {
+  console.log("Error "  + error);
+
+  assert(error == null);
+
+  assert(res.indexOf("AFFINITY RUN") !== -1);
+
+  console.log("!!!!!!!!RES = " + res);
+
+  TestUtils.testDone();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e77e76ac/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 fac9818..a9a5b9d 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
@@ -447,6 +447,16 @@ public class GridJettyRestHandler extends AbstractHandler {
                 break;
             }
 
+            case AFFINITY_RUN: {
+                System.out.println("!!!!!!!AFFINITY RUN");
+                RestComputeRequest restReq0 = new RestComputeRequest();
+
+                restReq0.function((String)params.get("func"));
+
+                restReq = restReq0;
+                break;
+            }
+
             default:
                 throw new IgniteCheckedException("Invalid command: " + cmd);
         }