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/24 12:06:49 UTC

[3/4] incubator-ignite git commit: #ignite-965: two commands EXECUTE_MAP_REDUCE_SCRIPT and RUN_SCRIPT.

#ignite-965: two commands EXECUTE_MAP_REDUCE_SCRIPT and RUN_SCRIPT.


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

Branch: refs/heads/ignite-965
Commit: 64d1108375db152c42d6208bb41e89855687c1df
Parents: 39ab970
Author: ivasilinets <iv...@gridgain.com>
Authored: Wed Jun 24 12:48:51 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Wed Jun 24 12:48:51 2015 +0300

----------------------------------------------------------------------
 .../processors/rest/GridRestCommand.java        | 11 ++-
 .../IgniteScriptingCommandHandler.java          | 40 ++++-------
 .../rest/request/RestComputeRequest.java        | 74 --------------------
 .../rest/request/RestComputeTaskRequest.java    | 74 --------------------
 .../request/RestMapReduceScriptRequest.java     | 74 ++++++++++++++++++++
 .../rest/request/RestRunScriptRequest.java      | 40 +++++++++++
 modules/nodejs/src/main/js/compute.js           | 18 +----
 .../ignite/internal/NodeJsComputeSelfTest.java  | 11 +--
 modules/nodejs/src/test/js/test-compute.js      | 30 ++------
 .../http/jetty/GridJettyRestHandler.java        | 13 ++--
 10 files changed, 144 insertions(+), 241 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/64d11083/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 f964cb4..4ee672d 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
@@ -96,14 +96,11 @@ public enum GridRestCommand {
     /** Quit. */
     QUIT("quit"),
 
-    /** Affinity run. */
-    AFFINITY_RUN("affscriptrun"),
+    /** Run script. */
+    RUN_SCRIPT("runscript"),
 
-    /** Affinity call. */
-    AFFINITY_CALL("affscriptcall"),
-
-    /** Execute task. */
-    EXECUTE_TASK("execscripttask");
+    /** Execute map reduce script. */
+    EXECUTE_MAP_REDUCE_SCRIPT("excmapreduce");
 
     /** Enum values. */
     private static final GridRestCommand[] VALS = values();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/64d11083/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/scripting/IgniteScriptingCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/scripting/IgniteScriptingCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/scripting/IgniteScriptingCommandHandler.java
index b402902..687f72a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/scripting/IgniteScriptingCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/scripting/IgniteScriptingCommandHandler.java
@@ -40,9 +40,9 @@ import static org.apache.ignite.internal.processors.rest.GridRestCommand.*;
  */
 public class IgniteScriptingCommandHandler extends GridRestCommandHandlerAdapter {
     /** Supported commands. */
-    private static final Collection<GridRestCommand> SUPPORTED_COMMANDS = U.sealList(EXECUTE_TASK,
-        AFFINITY_RUN,
-        AFFINITY_CALL);
+    private static final Collection<GridRestCommand> SUPPORTED_COMMANDS = U.sealList(
+        EXECUTE_MAP_REDUCE_SCRIPT,
+        RUN_SCRIPT);
 
     /**
      * @param ctx Context.
@@ -77,46 +77,30 @@ public class IgniteScriptingCommandHandler extends GridRestCommandHandlerAdapter
         assert SUPPORTED_COMMANDS.contains(req.command());
 
         switch (req.command()) {
-            case AFFINITY_RUN: {
-                assert req instanceof RestComputeRequest : "Invalid type of compute request.";
+            case RUN_SCRIPT: {
+                assert req instanceof RestRunScriptRequest : "Invalid type of run script request.";
 
-                final RestComputeRequest req0 = (RestComputeRequest) req;
+                final RestRunScriptRequest req0 = (RestRunScriptRequest) req;
 
-                ctx.grid().compute().affinityRun(req0.cacheName(), req0.key(), new IgniteRunnable() {
-                    @IgniteInstanceResource
-                    private Ignite ignite;
-
-                    @Override public void run() {
-                        ((IgniteKernal) ignite).context().scripting().invokeFunction(req0.function());
-                    }
-                });
-
-                return new GridFinishedFuture<>(new GridRestResponse());
-            }
-
-            case AFFINITY_CALL: {
-                assert req instanceof RestComputeRequest : "Invalid type of compute request.";
-
-                final RestComputeRequest req0 = (RestComputeRequest) req;
-
-                Object callRes = ctx.grid().compute().affinityCall(req0.cacheName(), req0.key(), new IgniteCallable<Object>() {
+                Object callRes = ctx.grid().compute().call(new IgniteCallable<Object>() {
                     @IgniteInstanceResource
                     private Ignite ignite;
 
                     @Override public Object call() {
-                        return ((IgniteKernal) ignite).context().scripting().invokeFunction(req0.function());
+                        return ((IgniteKernal) ignite).context().scripting().invokeFunction(req0.script());
                     }
                 });
 
                 return new GridFinishedFuture<>(new GridRestResponse(callRes));
             }
 
-            case  EXECUTE_TASK: {
-                assert req instanceof RestComputeTaskRequest : "Invalid type of compute task request.";
+            case EXECUTE_MAP_REDUCE_SCRIPT: {
+                assert req instanceof RestMapReduceScriptRequest :
+                    "Invalid type of execute map reduce script request.";
 
                 assert SUPPORTED_COMMANDS.contains(req.command());
 
-                final RestComputeTaskRequest req0 = (RestComputeTaskRequest) req;
+                final RestMapReduceScriptRequest req0 = (RestMapReduceScriptRequest) req;
 
                 Object execRes = ctx.grid().compute().execute(
                     new JsTask(req0.mapFunction(), req0.argument(), req0.reduceFunction(), ctx), null);

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

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/64d11083/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
deleted file mode 100644
index cf5c7c0..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/RestComputeTaskRequest.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.rest.request;
-
-/**
- * Compute task request.
- */
-public class RestComputeTaskRequest extends GridRestRequest {
-    /** Mapping tasks to nodes. */
-    private String mapFunc;
-
-    /** Function argument. */
-    private String arg;
-
-    /** Reduce function. */
-    private String reduceFunc;
-
-    /**
-     * @param reduceFunc Reduce function.
-     */
-    public void reduceFunction(String reduceFunc) {
-        this.reduceFunc = reduceFunc;
-    }
-
-    /**
-     * @return Reduce function.
-     */
-    public String reduceFunction() {
-        return reduceFunc;
-    }
-
-    /**
-     * @param mapFunc Map function.
-     */
-    public void mapFunction(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/64d11083/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/RestMapReduceScriptRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/RestMapReduceScriptRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/RestMapReduceScriptRequest.java
new file mode 100644
index 0000000..c21fcf4
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/RestMapReduceScriptRequest.java
@@ -0,0 +1,74 @@
+/*
+ * 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;
+
+/**
+ * Execute map reduce script request.
+ */
+public class RestMapReduceScriptRequest extends GridRestRequest {
+    /** Mapping tasks to nodes. */
+    private String mapFunc;
+
+    /** Function argument. */
+    private String arg;
+
+    /** Reduce function. */
+    private String reduceFunc;
+
+    /**
+     * @param reduceFunc Reduce function.
+     */
+    public void reduceFunction(String reduceFunc) {
+        this.reduceFunc = reduceFunc;
+    }
+
+    /**
+     * @return Reduce function.
+     */
+    public String reduceFunction() {
+        return reduceFunc;
+    }
+
+    /**
+     * @param mapFunc Map function.
+     */
+    public void mapFunction(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/64d11083/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/RestRunScriptRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/RestRunScriptRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/RestRunScriptRequest.java
new file mode 100644
index 0000000..42aba76
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/RestRunScriptRequest.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;
+
+/**
+ * Run script request.
+ */
+public class RestRunScriptRequest extends GridRestRequest {
+    /** Java script function. */
+    private String script;
+
+    /**
+     * @return Java script function.
+     */
+    public String script() {
+        return script;
+    }
+
+    /**
+     * @param script Java script function.
+     */
+    public void script(String script) {
+        this.script = script;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/64d11083/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 01361d5..c7a386e 100644
--- a/modules/nodejs/src/main/js/compute.js
+++ b/modules/nodejs/src/main/js/compute.js
@@ -30,23 +30,11 @@ function Compute(server) {
  * @this {Compute}
  * @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),
-        Server.pair("key", key), Server.pair("func", this._escape(runnable))], callback);
-}
-
-/**
- * @this {Compute}
- * @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),
+Compute.prototype.runScript = function(cacheName, key, runnable, callback) {
+    this._server.runCommand("runscript", [Server.pair("cacheName", cacheName),
         Server.pair("key", key), Server.pair("func", this._escape(runnable))], callback);
 }
 
@@ -64,7 +52,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("execscripttask", params, callback);
+    this._server.runCommand("excmapreduce", params, callback);
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/64d11083/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 5c4635d..82afe9b 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
@@ -41,15 +41,8 @@ public class NodeJsComputeSelfTest extends NodeJsAbstractTest {
     /**
      * @throws Exception If failed.
      */
-    public void testComputeAffinityRun() throws Exception {
-        runJsScript("testComputeAffinityRun");
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testComputeAffinityCall() throws Exception {
-        runJsScript("testComputeAffinityCall");
+    public void testComputeRunScript() throws Exception {
+        runJsScript("testComputeRunScript");
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/64d11083/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 a494f3e..13b2ecf 100644
--- a/modules/nodejs/src/test/js/test-compute.js
+++ b/modules/nodejs/src/test/js/test-compute.js
@@ -19,12 +19,8 @@ var TestUtils = require("./test-utils").TestUtils;
 
 var assert = require("assert");
 
-testComputeAffinityRun = function() {
-    TestUtils.startIgniteNode(onStart.bind(null, computeAffinityRun));
-}
-
-testComputeAffinityCall = function() {
-    TestUtils.startIgniteNode(onStart.bind(null, computeAffinityCall));
+testComputeRunScript = function() {
+    TestUtils.startIgniteNode(onStart.bind(null, computeRunScript));
 }
 
 testComputeExecute = function() {
@@ -43,25 +39,7 @@ function onStart(onPut, error, ignite) {
     cache.putAll(params, onPut.bind(null, ignite))
 }
 
-function computeAffinityRun(ignite, error) {
-    var comp = ignite.compute();
-
-    var f = function () {
-        println("Hello world!");
-
-        ignite.hello();
-    }
-
-    function onEnd(error) {
-        assert(error == null);
-
-        TestUtils.testDone();
-    }
-
-    comp.affinityRun("mycache", "key999", f, onEnd.bind(null));
-}
-
-function computeAffinityCall(ignite, error) {
+function computeRunScript(ignite, error) {
     var comp = ignite.compute();
 
     var f = function () {
@@ -75,7 +53,7 @@ function computeAffinityCall(ignite, error) {
         TestUtils.testDone();
     }
 
-    comp.affinityCall("mycache", "key999", f, onEnd.bind(null));
+    comp.runScript("mycache", "key999", f, onEnd.bind(null));
 }
 
 function computeExecute(error, ignite) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/64d11083/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 0d961be..faffbf1 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,21 +447,18 @@ public class GridJettyRestHandler extends AbstractHandler {
                 break;
             }
 
-            case AFFINITY_RUN:
-            case AFFINITY_CALL: {
-                RestComputeRequest restReq0 = new RestComputeRequest();
+            case RUN_SCRIPT: {
+                RestRunScriptRequest restReq0 = new RestRunScriptRequest();
 
-                restReq0.function((String)params.get("func"));
-                restReq0.cacheName((String)params.get("cacheName"));
-                restReq0.key(params.get("key"));
+                restReq0.script((String) params.get("func"));
 
                 restReq = restReq0;
 
                 break;
             }
 
-            case EXECUTE_TASK: {
-                RestComputeTaskRequest restReq0 = new RestComputeTaskRequest();
+            case EXECUTE_MAP_REDUCE_SCRIPT: {
+                RestMapReduceScriptRequest restReq0 = new RestMapReduceScriptRequest();
 
                 restReq0.mapFunction((String) params.get("map"));
                 restReq0.argument((String)params.get("arg"));