You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by iv...@apache.org on 2015/07/05 21:25:53 UTC

incubator-ignite git commit: #ignite-964: move IgniteScriptingCommandHandler to core module.

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-964-1 b8596b601 -> a8873e3dd


#ignite-964: move IgniteScriptingCommandHandler to core module.


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

Branch: refs/heads/ignite-964-1
Commit: a8873e3dd44fdb9362d7e516e9a24b223afea089
Parents: b8596b6
Author: ivasilinets <iv...@gridgain.com>
Authored: Sun Jul 5 22:25:45 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Sun Jul 5 22:25:45 2015 +0300

----------------------------------------------------------------------
 .../processors/rest/GridRestProcessor.java      |  28 +-
 .../IgniteScriptingCommandHandler.java          | 336 +++++++++++++++++++
 .../jetty/IgniteScriptingCommandHandler.java    | 336 -------------------
 3 files changed, 338 insertions(+), 362 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a8873e3d/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 a27327c..49120ed 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
@@ -26,6 +26,7 @@ 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.query.*;
 import org.apache.ignite.internal.processors.rest.handlers.datastructures.*;
+import org.apache.ignite.internal.processors.rest.handlers.scripting.IgniteScriptingCommandHandler;
 import org.apache.ignite.internal.processors.rest.handlers.task.*;
 import org.apache.ignite.internal.processors.rest.handlers.top.*;
 import org.apache.ignite.internal.processors.rest.handlers.version.*;
@@ -296,7 +297,7 @@ public class GridRestProcessor extends GridProcessorAdapter {
             addHandler(new GridVersionNameCommandHandler(ctx));
             addHandler(new DataStructuresCommandHandler(ctx));
             addHandler(new QueryCommandHandler(ctx));
-            addScriptingHandler();
+            addHandler(new IgniteScriptingCommandHandler(ctx));
         }
     }
 
@@ -655,31 +656,6 @@ public class GridRestProcessor extends GridProcessorAdapter {
     }
 
     /**
-     * Add scripting handler if exist in classpath.
-     *
-     * @throws IgniteCheckedException In case of error.
-     */
-    private void addScriptingHandler() throws IgniteCheckedException {
-        try {
-            Class<?> cls1 = Class.forName(HTTP_SCRIPTING_CLS);
-
-            Constructor<?> ctor1 = cls1.getConstructor(GridKernalContext.class);
-
-            GridRestCommandHandlerAdapter handl1 = (GridRestCommandHandlerAdapter)ctor1.newInstance(ctx);
-
-            addHandler(handl1);
-        }
-        catch (ClassNotFoundException ignored) {
-            if (log.isDebugEnabled())
-                log.debug("Failed to add scripting handler (consider adding ignite-rest-http " +
-                    "module to classpath).");
-        }
-        catch (NoSuchMethodException | InvocationTargetException | InstantiationException | IllegalAccessException e) {
-            throw new IgniteCheckedException("Failed to initialize HTTP REST protocol.", e);
-        }
-    }
-
-    /**
      * @return Client configuration.
      */
     private ConnectorConfiguration config() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a8873e3d/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
new file mode 100644
index 0000000..a792c2a
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/scripting/IgniteScriptingCommandHandler.java
@@ -0,0 +1,336 @@
+/*
+ * 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.scripting;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.compute.*;
+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.handlers.scripting.*;
+import org.apache.ignite.internal.processors.rest.request.*;
+import org.apache.ignite.internal.processors.scripting.*;
+import org.apache.ignite.internal.util.future.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.resources.*;
+import org.jetbrains.annotations.*;
+
+import java.util.*;
+import java.util.concurrent.*;
+
+import static org.apache.ignite.internal.processors.rest.GridRestCommand.*;
+
+/**
+ * Compute task command handler.
+ */
+public class IgniteScriptingCommandHandler extends GridRestCommandHandlerAdapter {
+    /** Supported commands. */
+    private static final Collection<GridRestCommand> SUPPORTED_COMMANDS = U.sealList(
+        EXECUTE_MAP_REDUCE_SCRIPT,
+        RUN_SCRIPT);
+
+    /** Emit result. */
+    private IgniteJsEmitResult emitRes;
+
+    /**
+     * @param ctx Context.
+     */
+    public IgniteScriptingCommandHandler(GridKernalContext ctx) {
+        super(ctx);
+
+        try {
+            IgniteScriptProcessor script = ctx.scripting();
+
+            String emitFunction = "function emit(f, args, nodeId) {" +
+                "__emitResult.add(f.toString(), args, nodeId);}";
+
+            script.addEngineFunction(emitFunction);
+
+            emitRes = new IgniteJsEmitResult();
+
+            script.addBinding("__emitResult", emitRes);
+
+
+            script.addBinding("ignite", new NodeJSIgnite(ctx.grid()));
+        }
+        catch (IgniteCheckedException e) {
+            ctx.log().error(e.getMessage());
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<GridRestCommand> supportedCommands() {
+        return SUPPORTED_COMMANDS;
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteInternalFuture<GridRestResponse> handleAsync(GridRestRequest req) {
+        assert req != null;
+
+        assert SUPPORTED_COMMANDS.contains(req.command());
+
+        switch (req.command()) {
+            case RUN_SCRIPT: {
+                assert req instanceof RestRunScriptRequest : "Invalid type of run script request.";
+
+                return ctx.closure().callLocalSafe(
+                        new RunScriptCallable(ctx, (RestRunScriptRequest) req), false);
+            }
+
+            case EXECUTE_MAP_REDUCE_SCRIPT: {
+                assert req instanceof RestMapReduceScriptRequest :
+                    "Invalid type of execute map reduce script request.";
+
+                return ctx.closure().callLocalSafe(
+                    new MapReduceCallable(ctx, (RestMapReduceScriptRequest)req, emitRes));
+            }
+        }
+
+        return new GridFinishedFuture<>();
+    }
+
+    /**
+     * JS Compute Task.
+     */
+    private static class JsTask extends ComputeTaskAdapter<String, Object> {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** Mapping function. */
+        private String mapFunc;
+
+        /** Reduce function. */
+        private String reduceFunc;
+
+        /** Kernal context. */
+        private GridKernalContext ctx;
+
+        /** Map function argument. */
+        private Object arg;
+
+        /** Emit results. */
+        private IgniteJsEmitResult emitRes;
+
+        /**
+         * @param mapFunc Map function.
+         * @param arg Map function argument.
+         * @param reduceFunc Reduce function.
+         * @param ctx Kernal context.
+         */
+        public JsTask(String mapFunc, Object arg, String reduceFunc, GridKernalContext ctx, IgniteJsEmitResult emitRes) {
+            this.mapFunc = mapFunc;
+            this.reduceFunc = reduceFunc;
+            this.arg = arg;
+            this.ctx = ctx;
+            this.emitRes = emitRes;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> nodes, String arg) {
+            try {
+                Map<ComputeJob, ClusterNode> map = new HashMap<>();
+
+                ctx.scripting().invokeFunction(mapFunc, nodes.toArray(new ClusterNode[nodes.size()]), this.arg);
+
+                List<T3<Object, Object, Object>> jsMapRes = emitRes.getEmitResult();
+
+                for (T3<Object, Object, Object> task : jsMapRes) {
+                    map.put(new JsCallFunctionJob((String)task.get1(), task.get2()),
+                        (ClusterNode)task.get3());
+                }
+
+                return map;
+            }
+            catch (IgniteCheckedException e) {
+                throw U.convertException(e);
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Nullable @Override public Object reduce(List<ComputeJobResult> results) {
+            try {
+                Object[] data = new Object[results.size()];
+
+                for (int i = 0; i < results.size(); ++i) {
+                    IgniteException err = results.get(i).getException();
+
+                    if (err != null)
+                        return new GridRestResponse(GridRestResponse.STATUS_FAILED, err.getMessage());
+
+                    data[i] = results.get(i).getData();
+                }
+
+                return ctx.scripting().invokeFunction(reduceFunc, data, null);
+            }
+            catch (IgniteCheckedException e) {
+                throw U.convertException(e);
+            }
+        }
+    }
+
+    /**
+     * Js call function job.
+     */
+    private static class JsCallFunctionJob extends ComputeJobAdapter {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** Function to call. */
+        private String func;
+
+        /** Function argument. */
+        private Object argv;
+
+        /** Ignite instance. */
+        @IgniteInstanceResource
+        private Ignite ignite;
+
+        /**
+         * @param func Function to call.
+         * @param argv Function argument.
+         */
+        public JsCallFunctionJob(String func, Object argv) {
+            this.func = func;
+
+            this.argv = RestJSONCacheObject.convertToRestObject(
+                JSONCacheObject.toSimpleObject(argv));
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object execute() throws IgniteException {
+            try {
+                return ((IgniteKernal)ignite).context().scripting().invokeFunction(func, argv, null);
+            }
+            catch (IgniteCheckedException e) {
+                throw U.convertException(e);
+            }
+            catch (Exception e) {
+                throw new IgniteException(e);
+            }
+        }
+    }
+
+    /**
+     * Call java script function.
+     */
+    private static class JsFunctionCallable implements IgniteCallable<Object> {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** Function to call. */
+        private String func;
+
+        /** Function argument. */
+        private Object arg;
+
+        /** Ignite instance. */
+        @IgniteInstanceResource
+        private Ignite ignite;
+
+        /**
+         * @param func Function to call.
+         * @param arg Function argument.
+         */
+        public JsFunctionCallable(String func, Object arg) {
+            this.func = func;
+            this.arg = arg;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object call() {
+            try {
+                return ((IgniteKernal)ignite).context().scripting().invokeFunction(func, arg);
+            }
+            catch (IgniteCheckedException e) {
+                throw U.convertException(e);
+            }
+        }
+    }
+
+    /**
+     * Run script callable.
+     */
+    private static class RunScriptCallable implements Callable<GridRestResponse> {
+        /** Kernal context. */
+        private GridKernalContext ctx;
+
+        /** Run script request. */
+        private RestRunScriptRequest req;
+
+        /**
+         * @param ctx Kernal context.
+         * @param req Run script request.
+         */
+        public RunScriptCallable(GridKernalContext ctx, RestRunScriptRequest req) {
+            this.ctx = ctx;
+            this.req = req;
+        }
+
+        /** {@inheritDoc} */
+        @Override public GridRestResponse call() throws Exception {
+            try {
+                return new GridRestResponse(ctx.grid().compute().call(
+                    new JsFunctionCallable(req.script(), req.argument())));
+            }
+            catch (Exception e) {
+                return new GridRestResponse(GridRestResponse.STATUS_FAILED, e.getMessage());
+            }
+        }
+    }
+
+    /**
+     * Map reduce callable.
+     */
+    private static class MapReduceCallable implements Callable<GridRestResponse> {
+        /** Kernal context. */
+        private GridKernalContext ctx;
+
+        /** Run script request. */
+        private RestMapReduceScriptRequest req;
+
+        /** Emit results. */
+        IgniteJsEmitResult emitRes;
+
+        /**
+         * @param ctx Kernal context.
+         * @param req Run script request.
+         * @param emitRes Emit function results.
+         */
+        public MapReduceCallable(GridKernalContext ctx, RestMapReduceScriptRequest req,
+            IgniteJsEmitResult emitRes) {
+            this.ctx = ctx;
+            this.req = req;
+            this.emitRes = emitRes;
+        }
+
+        /** {@inheritDoc} */
+        @Override public GridRestResponse call() throws Exception {
+            try {
+                return new GridRestResponse(ctx.grid().compute().execute(
+                    new JsTask(req.mapFunction(), req.argument(), req.reduceFunction(), ctx, emitRes),
+                    null));
+            }
+            catch (Exception e) {
+                return new GridRestResponse(GridRestResponse.STATUS_FAILED, e.getMessage());
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a8873e3d/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/IgniteScriptingCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/IgniteScriptingCommandHandler.java b/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/IgniteScriptingCommandHandler.java
deleted file mode 100644
index 400f421..0000000
--- a/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/IgniteScriptingCommandHandler.java
+++ /dev/null
@@ -1,336 +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.protocols.http.jetty;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cluster.*;
-import org.apache.ignite.compute.*;
-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.handlers.scripting.*;
-import org.apache.ignite.internal.processors.rest.request.*;
-import org.apache.ignite.internal.processors.scripting.*;
-import org.apache.ignite.internal.util.future.*;
-import org.apache.ignite.internal.util.typedef.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.lang.*;
-import org.apache.ignite.resources.*;
-import org.jetbrains.annotations.*;
-
-import java.util.*;
-import java.util.concurrent.*;
-
-import static org.apache.ignite.internal.processors.rest.GridRestCommand.*;
-
-/**
- * Compute task command handler.
- */
-public class IgniteScriptingCommandHandler extends GridRestCommandHandlerAdapter {
-    /** Supported commands. */
-    private static final Collection<GridRestCommand> SUPPORTED_COMMANDS = U.sealList(
-        EXECUTE_MAP_REDUCE_SCRIPT,
-        RUN_SCRIPT);
-
-    /** Emit result. */
-    private IgniteJsEmitResult emitRes;
-
-    /**
-     * @param ctx Context.
-     */
-    public IgniteScriptingCommandHandler(GridKernalContext ctx) {
-        super(ctx);
-
-        try {
-            IgniteScriptProcessor script = ctx.scripting();
-
-            String emitFunction = "function emit(f, args, nodeId) {" +
-                "__emitResult.add(f.toString(), args, nodeId);}";
-
-            script.addEngineFunction(emitFunction);
-
-            emitRes = new IgniteJsEmitResult();
-
-            script.addBinding("__emitResult", emitRes);
-
-
-            script.addBinding("ignite", new NodeJSIgnite(ctx.grid()));
-        }
-        catch (IgniteCheckedException e) {
-            ctx.log().error(e.getMessage());
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public Collection<GridRestCommand> supportedCommands() {
-        return SUPPORTED_COMMANDS;
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<GridRestResponse> handleAsync(GridRestRequest req) {
-        assert req != null;
-
-        assert SUPPORTED_COMMANDS.contains(req.command());
-
-        switch (req.command()) {
-            case RUN_SCRIPT: {
-                assert req instanceof RestRunScriptRequest : "Invalid type of run script request.";
-
-                return ctx.closure().callLocalSafe(
-                        new RunScriptCallable(ctx, (RestRunScriptRequest) req), false);
-            }
-
-            case EXECUTE_MAP_REDUCE_SCRIPT: {
-                assert req instanceof RestMapReduceScriptRequest :
-                    "Invalid type of execute map reduce script request.";
-
-                return ctx.closure().callLocalSafe(
-                    new MapReduceCallable(ctx, (RestMapReduceScriptRequest)req, emitRes));
-            }
-        }
-
-        return new GridFinishedFuture<>();
-    }
-
-    /**
-     * JS Compute Task.
-     */
-    private static class JsTask extends ComputeTaskAdapter<String, Object> {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /** Mapping function. */
-        private String mapFunc;
-
-        /** Reduce function. */
-        private String reduceFunc;
-
-        /** Kernal context. */
-        private GridKernalContext ctx;
-
-        /** Map function argument. */
-        private Object arg;
-
-        /** Emit results. */
-        private IgniteJsEmitResult emitRes;
-
-        /**
-         * @param mapFunc Map function.
-         * @param arg Map function argument.
-         * @param reduceFunc Reduce function.
-         * @param ctx Kernal context.
-         */
-        public JsTask(String mapFunc, Object arg, String reduceFunc, GridKernalContext ctx, IgniteJsEmitResult emitRes) {
-            this.mapFunc = mapFunc;
-            this.reduceFunc = reduceFunc;
-            this.arg = arg;
-            this.ctx = ctx;
-            this.emitRes = emitRes;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> nodes, String arg) {
-            try {
-                Map<ComputeJob, ClusterNode> map = new HashMap<>();
-
-                ctx.scripting().invokeFunction(mapFunc, nodes.toArray(new ClusterNode[nodes.size()]), this.arg);
-
-                List<T3<Object, Object, Object>> jsMapRes = emitRes.getEmitResult();
-
-                for (T3<Object, Object, Object> task : jsMapRes) {
-                    map.put(new JsCallFunctionJob((String)task.get1(), task.get2()),
-                        (ClusterNode)task.get3());
-                }
-
-                return map;
-            }
-            catch (IgniteCheckedException e) {
-                throw U.convertException(e);
-            }
-        }
-
-        /** {@inheritDoc} */
-        @Nullable @Override public Object reduce(List<ComputeJobResult> results) {
-            try {
-                Object[] data = new Object[results.size()];
-
-                for (int i = 0; i < results.size(); ++i) {
-                    IgniteException err = results.get(i).getException();
-
-                    if (err != null)
-                        return new GridRestResponse(GridRestResponse.STATUS_FAILED, err.getMessage());
-
-                    data[i] = results.get(i).getData();
-                }
-
-                return ctx.scripting().invokeFunction(reduceFunc, data, null);
-            }
-            catch (IgniteCheckedException e) {
-                throw U.convertException(e);
-            }
-        }
-    }
-
-    /**
-     * Js call function job.
-     */
-    private static class JsCallFunctionJob extends ComputeJobAdapter {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /** Function to call. */
-        private String func;
-
-        /** Function argument. */
-        private Object argv;
-
-        /** Ignite instance. */
-        @IgniteInstanceResource
-        private Ignite ignite;
-
-        /**
-         * @param func Function to call.
-         * @param argv Function argument.
-         */
-        public JsCallFunctionJob(String func, Object argv) {
-            this.func = func;
-
-            this.argv = RestJSONCacheObject.convertToRestObject(
-                JSONCacheObject.toSimpleObject(argv));
-        }
-
-        /** {@inheritDoc} */
-        @Override public Object execute() throws IgniteException {
-            try {
-                return ((IgniteKernal)ignite).context().scripting().invokeFunction(func, argv, null);
-            }
-            catch (IgniteCheckedException e) {
-                throw U.convertException(e);
-            }
-            catch (Exception e) {
-                throw new IgniteException(e);
-            }
-        }
-    }
-
-    /**
-     * Call java script function.
-     */
-    private static class JsFunctionCallable implements IgniteCallable<Object> {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /** Function to call. */
-        private String func;
-
-        /** Function argument. */
-        private Object arg;
-
-        /** Ignite instance. */
-        @IgniteInstanceResource
-        private Ignite ignite;
-
-        /**
-         * @param func Function to call.
-         * @param arg Function argument.
-         */
-        public JsFunctionCallable(String func, Object arg) {
-            this.func = func;
-            this.arg = arg;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Object call() {
-            try {
-                return ((IgniteKernal)ignite).context().scripting().invokeFunction(func, arg);
-            }
-            catch (IgniteCheckedException e) {
-                throw U.convertException(e);
-            }
-        }
-    }
-
-    /**
-     * Run script callable.
-     */
-    private static class RunScriptCallable implements Callable<GridRestResponse> {
-        /** Kernal context. */
-        private GridKernalContext ctx;
-
-        /** Run script request. */
-        private RestRunScriptRequest req;
-
-        /**
-         * @param ctx Kernal context.
-         * @param req Run script request.
-         */
-        public RunScriptCallable(GridKernalContext ctx, RestRunScriptRequest req) {
-            this.ctx = ctx;
-            this.req = req;
-        }
-
-        /** {@inheritDoc} */
-        @Override public GridRestResponse call() throws Exception {
-            try {
-                return new GridRestResponse(ctx.grid().compute().call(
-                    new JsFunctionCallable(req.script(), req.argument())));
-            }
-            catch (Exception e) {
-                return new GridRestResponse(GridRestResponse.STATUS_FAILED, e.getMessage());
-            }
-        }
-    }
-
-    /**
-     * Map reduce callable.
-     */
-    private static class MapReduceCallable implements Callable<GridRestResponse> {
-        /** Kernal context. */
-        private GridKernalContext ctx;
-
-        /** Run script request. */
-        private RestMapReduceScriptRequest req;
-
-        /** Emit results. */
-        IgniteJsEmitResult emitRes;
-
-        /**
-         * @param ctx Kernal context.
-         * @param req Run script request.
-         * @param emitRes Emit function results.
-         */
-        public MapReduceCallable(GridKernalContext ctx, RestMapReduceScriptRequest req,
-            IgniteJsEmitResult emitRes) {
-            this.ctx = ctx;
-            this.req = req;
-            this.emitRes = emitRes;
-        }
-
-        /** {@inheritDoc} */
-        @Override public GridRestResponse call() throws Exception {
-            try {
-                return new GridRestResponse(ctx.grid().compute().execute(
-                    new JsTask(req.mapFunction(), req.argument(), req.reduceFunction(), ctx, emitRes),
-                    null));
-            }
-            catch (Exception e) {
-                return new GridRestResponse(GridRestResponse.STATUS_FAILED, e.getMessage());
-            }
-        }
-    }
-}