You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vo...@apache.org on 2015/10/22 11:09:18 UTC

[06/50] [abbrv] ignite git commit: ignite-1168 Added support for metadata, scan commands in rest.

http://git-wip-us.apache.org/repos/asf/ignite/blob/12235254/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/top/GridTopologyCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/top/GridTopologyCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/top/GridTopologyCommandHandler.java
index aee8a1f..3b36c6a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/top/GridTopologyCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/top/GridTopologyCommandHandler.java
@@ -27,11 +27,11 @@ import java.util.Iterator;
 import java.util.Map;
 import java.util.UUID;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cluster.ClusterMetrics;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.IgniteInternalFuture;
-import org.apache.ignite.internal.processors.cache.GridCacheAttributes;
 import org.apache.ignite.internal.processors.port.GridPortRecord;
 import org.apache.ignite.internal.processors.rest.GridRestCommand;
 import org.apache.ignite.internal.processors.rest.GridRestProtocol;
@@ -44,6 +44,7 @@ import org.apache.ignite.internal.processors.rest.request.GridRestTopologyReques
 import org.apache.ignite.internal.util.future.GridFinishedFuture;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.P1;
+import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.spi.IgnitePortProtocol;
@@ -194,24 +195,26 @@ public class GridTopologyCommandHandler extends GridRestCommandHandlerAdapter {
         nodeBean.setTcpAddresses(nonEmptyList(node.<Collection<String>>attribute(ATTR_REST_TCP_ADDRS)));
         nodeBean.setTcpHostNames(nonEmptyList(node.<Collection<String>>attribute(ATTR_REST_TCP_HOST_NAMES)));
 
-        GridCacheAttributes[] caches = node.attribute(ATTR_CACHE);
+        Map<String, CacheMode> nodeCaches = ctx.discovery().nodeCaches(node);
 
-        if (!F.isEmpty(caches)) {
-            Map<String, String> cacheMap = new HashMap<>();
+        Map<String, String> cacheMap = U.newHashMap(nodeCaches.size());
 
-            for (GridCacheAttributes cacheAttr : caches) {
-                if (ctx.cache().systemCache(cacheAttr.cacheName()))
-                    continue;
+        for (Map.Entry<String, CacheMode> cache : nodeCaches.entrySet()) {
+            String cacheName = cache.getKey();
 
-                if (cacheAttr.cacheName() != null)
-                    cacheMap.put(cacheAttr.cacheName(), cacheAttr.cacheMode().toString());
-                else
-                    nodeBean.setDefaultCacheMode(cacheAttr.cacheMode().toString());
-            }
+            if (CU.isSystemCache(cacheName) || CU.isIgfsCache(ctx.config(), cacheName))
+                continue;
+
+            String mode = cache.getValue().toString();
 
-            nodeBean.setCaches(cacheMap);
+            if (cacheName != null)
+                cacheMap.put(cacheName, mode);
+            else
+                nodeBean.setDefaultCacheMode(mode);
         }
 
+        nodeBean.setCaches(cacheMap);
+
         if (mtr) {
             ClusterMetrics metrics = node.metrics();
 
@@ -304,6 +307,135 @@ public class GridTopologyCommandHandler extends GridRestCommandHandlerAdapter {
     }
 
     /**
+     * Creates node bean out of grid node. Notice that cache attribute is handled separately.
+     *
+     * @param node Grid node.
+     * @param mtr {@code true} to add metrics.
+     * @param attr {@code true} to add attributes.
+     * @return Grid Node bean.
+     */
+    private GridClientNodeBean createNodeBean(ClusterNode node, boolean mtr, boolean attr) {
+        assert node != null;
+
+        GridClientNodeBean nodeBean = new GridClientNodeBean();
+
+        nodeBean.setNodeId(node.id());
+        nodeBean.setConsistentId(node.consistentId());
+        nodeBean.setTcpPort(attribute(node, ATTR_REST_TCP_PORT, 0));
+
+        nodeBean.setTcpAddresses(nonEmptyList(node.<Collection<String>>attribute(ATTR_REST_TCP_ADDRS)));
+        nodeBean.setTcpHostNames(nonEmptyList(node.<Collection<String>>attribute(ATTR_REST_TCP_HOST_NAMES)));
+
+        Map<String, CacheMode> nodeCaches = ctx.discovery().nodeCaches(node);
+
+        Map<String, String> cacheMap = U.newHashMap(nodeCaches.size());
+
+        for (Map.Entry<String, CacheMode> cache : nodeCaches.entrySet()) {
+            String cacheName = cache.getKey();
+
+            if (CU.isSystemCache(cacheName) || CU.isIgfsCache(ctx.config(), cacheName))
+                continue;
+
+            String mode = cache.getValue().toString();
+
+            if (cacheName != null)
+                cacheMap.put(cacheName, mode);
+            else
+                nodeBean.setDefaultCacheMode(mode);
+        }
+
+        nodeBean.setCaches(cacheMap);
+
+        if (mtr) {
+            ClusterMetrics metrics = node.metrics();
+
+            GridClientNodeMetricsBean metricsBean = new GridClientNodeMetricsBean();
+
+            metricsBean.setStartTime(metrics.getStartTime());
+            metricsBean.setAverageActiveJobs(metrics.getAverageActiveJobs());
+            metricsBean.setAverageCancelledJobs(metrics.getAverageCancelledJobs());
+            metricsBean.setAverageCpuLoad(metrics.getAverageCpuLoad());
+            metricsBean.setAverageJobExecuteTime(metrics.getAverageJobExecuteTime());
+            metricsBean.setAverageJobWaitTime(metrics.getAverageJobWaitTime());
+            metricsBean.setAverageRejectedJobs(metrics.getAverageRejectedJobs());
+            metricsBean.setAverageWaitingJobs(metrics.getAverageWaitingJobs());
+            metricsBean.setCurrentActiveJobs(metrics.getCurrentActiveJobs());
+            metricsBean.setCurrentCancelledJobs(metrics.getCurrentCancelledJobs());
+            metricsBean.setCurrentCpuLoad(metrics.getCurrentCpuLoad());
+            metricsBean.setCurrentGcCpuLoad(metrics.getCurrentGcCpuLoad());
+            metricsBean.setCurrentDaemonThreadCount(metrics.getCurrentDaemonThreadCount());
+            metricsBean.setCurrentIdleTime(metrics.getCurrentIdleTime());
+            metricsBean.setCurrentJobExecuteTime(metrics.getCurrentJobExecuteTime());
+            metricsBean.setCurrentJobWaitTime(metrics.getCurrentJobWaitTime());
+            metricsBean.setCurrentRejectedJobs(metrics.getCurrentRejectedJobs());
+            metricsBean.setCurrentThreadCount(metrics.getCurrentThreadCount());
+            metricsBean.setCurrentWaitingJobs(metrics.getCurrentWaitingJobs());
+            metricsBean.setHeapMemoryCommitted(metrics.getHeapMemoryCommitted());
+            metricsBean.setHeapMemoryInitialized(metrics.getHeapMemoryInitialized());
+            metricsBean.setHeapMemoryMaximum(metrics.getHeapMemoryMaximum());
+            metricsBean.setHeapMemoryUsed(metrics.getHeapMemoryUsed());
+            metricsBean.setLastDataVersion(metrics.getLastDataVersion());
+            metricsBean.setLastUpdateTime(metrics.getLastUpdateTime());
+            metricsBean.setMaximumActiveJobs(metrics.getMaximumActiveJobs());
+            metricsBean.setMaximumCancelledJobs(metrics.getMaximumCancelledJobs());
+            metricsBean.setMaximumJobExecuteTime(metrics.getMaximumJobExecuteTime());
+            metricsBean.setMaximumJobWaitTime(metrics.getMaximumJobWaitTime());
+            metricsBean.setMaximumRejectedJobs(metrics.getMaximumRejectedJobs());
+            metricsBean.setMaximumThreadCount(metrics.getMaximumThreadCount());
+            metricsBean.setMaximumWaitingJobs(metrics.getMaximumWaitingJobs());
+            metricsBean.setNodeStartTime(metrics.getNodeStartTime());
+            metricsBean.setNonHeapMemoryCommitted(metrics.getNonHeapMemoryCommitted());
+            metricsBean.setNonHeapMemoryInitialized(metrics.getNonHeapMemoryInitialized());
+            metricsBean.setNonHeapMemoryMaximum(metrics.getNonHeapMemoryMaximum());
+            metricsBean.setNonHeapMemoryUsed(metrics.getNonHeapMemoryUsed());
+            metricsBean.setStartTime(metrics.getStartTime());
+            metricsBean.setTotalCancelledJobs(metrics.getTotalCancelledJobs());
+            metricsBean.setTotalCpus(metrics.getTotalCpus());
+            metricsBean.setTotalExecutedJobs(metrics.getTotalExecutedJobs());
+            metricsBean.setTotalIdleTime(metrics.getTotalIdleTime());
+            metricsBean.setTotalRejectedJobs(metrics.getTotalRejectedJobs());
+            metricsBean.setTotalStartedThreadCount(metrics.getTotalStartedThreadCount());
+            metricsBean.setTotalExecutedTasks(metrics.getTotalExecutedTasks());
+            metricsBean.setSentMessagesCount(metrics.getSentMessagesCount());
+            metricsBean.setSentBytesCount(metrics.getSentBytesCount());
+            metricsBean.setReceivedMessagesCount(metrics.getReceivedMessagesCount());
+            metricsBean.setReceivedBytesCount(metrics.getReceivedBytesCount());
+            metricsBean.setUpTime(metrics.getUpTime());
+
+            nodeBean.setMetrics(metricsBean);
+        }
+
+        if (attr) {
+            Map<String, Object> attrs = new HashMap<>(node.attributes());
+
+            attrs.remove(ATTR_CACHE);
+            attrs.remove(ATTR_TX_CONFIG);
+            attrs.remove(ATTR_SECURITY_SUBJECT);
+            attrs.remove(ATTR_SECURITY_CREDENTIALS);
+
+            for (Iterator<Map.Entry<String, Object>> i = attrs.entrySet().iterator(); i.hasNext();) {
+                Map.Entry<String, Object> e = i.next();
+
+                if (!e.getKey().startsWith("org.apache.ignite.") && !e.getKey().startsWith("plugins.") &&
+                    System.getProperty(e.getKey()) == null) {
+                    i.remove();
+
+                    continue;
+                }
+
+                if (e.getValue() != null) {
+                  if (e.getValue().getClass().isEnum() || e.getValue() instanceof InetAddress)
+                      e.setValue(e.getValue().toString());
+                  else if (e.getValue().getClass().isArray())
+                      i.remove();
+                }
+            }
+
+            nodeBean.setAttributes(attrs);
+        }
+
+        return nodeBean;
+    }    /**
      * @param col Collection;
      * @return Non-empty list.
      */
@@ -345,4 +477,4 @@ public class GridTopologyCommandHandler extends GridRestCommandHandlerAdapter {
     @Override public String toString() {
         return S.toString(GridTopologyCommandHandler.class, this);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/12235254/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/RestQueryRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/RestQueryRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/RestQueryRequest.java
new file mode 100644
index 0000000..a719776
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/RestQueryRequest.java
@@ -0,0 +1,175 @@
+/*
+ *
+ *  * 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;
+
+/**
+ * Sql query request.
+ */
+public class RestQueryRequest extends GridRestRequest {
+    /** Sql query. */
+    private String sqlQry;
+
+    /** Sql query arguments. */
+    private Object[] args;
+
+    /** Page size. */
+    private Integer pageSize;
+
+    /** Cache name. */
+    private String cacheName;
+
+    /** Query id. */
+    private Long qryId;
+
+    /** Query type name. */
+    private String typeName;
+
+    /** Predicate class name for scan query. */
+    private String className;
+
+    /** Query type. */
+    private QueryType type;
+
+    /**
+     * @param sqlQry Sql query.
+     */
+    public void sqlQuery(String sqlQry) {
+        this.sqlQry = sqlQry;
+    }
+
+    /**
+     * @return Sql query.
+     */
+    public String sqlQuery() {
+        return sqlQry;
+    }
+
+    /**
+     * @param args Sql query arguments.
+     */
+    public void arguments(Object[] args) {
+        this.args = args;
+    }
+
+    /**
+     * @return Sql query arguments.
+     */
+    public Object[] arguments() {
+        return args;
+    }
+
+    /**
+     * @param pageSize Page size.
+     */
+    public void pageSize(Integer pageSize) {
+        this.pageSize = pageSize;
+    }
+
+    /**
+     * @return Page size.
+     */
+    public int pageSize() {
+        return pageSize;
+    }
+
+    /**
+     * @param cacheName Cache name.
+     */
+    public void cacheName(String cacheName) {
+        this.cacheName = cacheName;
+    }
+
+    /**
+     * @return Cache name.
+     */
+    public String cacheName() {
+        return cacheName;
+    }
+
+    /**
+     * @param id Query id.
+     */
+    public void queryId(Long id) {
+        this.qryId = id;
+    }
+
+    /**
+     * @return Query id.
+     */
+    public Long queryId() {
+        return qryId;
+    }
+
+    /**
+     * @param typeName Query type name.
+     */
+    public void typeName(String typeName) {
+        this.typeName = typeName;
+    }
+
+    /**
+     * @return Query type name.
+     */
+    public String typeName() {
+        return typeName;
+    }
+
+    /**
+     * @return Predicate class name for scan query.
+     */
+    public String className() {
+        return className;
+    }
+
+    /**
+     * @param className Predicate class name for scan query.
+     */
+    public void className(String className) {
+        this.className = className;
+    }
+
+    /**
+     * @param type Query type.
+     */
+    public void queryType(QueryType type) {
+        this.type = type;
+    }
+
+    /**
+     * @return Query type.
+     */
+    public QueryType queryType() {
+        return type;
+    }
+
+    /**
+     * Supported query types.
+     */
+    public enum QueryType {
+        /** Sql query. */
+        SQL,
+
+        /** Sql fields query. */
+        SQL_FIELDS,
+
+        /** Scan query. */
+        SCAN
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/12235254/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/RestSqlQueryRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/RestSqlQueryRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/RestSqlQueryRequest.java
deleted file mode 100644
index b6ab339..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/RestSqlQueryRequest.java
+++ /dev/null
@@ -1,125 +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;
-
-/**
- * Sql query request.
- */
-public class RestSqlQueryRequest extends GridRestRequest {
-    /** Sql query. */
-    private String sqlQry;
-
-    /** Sql query arguments. */
-    private Object[] args;
-
-    /** Page size. */
-    private Integer pageSize;
-
-    /** Cache name. */
-    private String cacheName;
-
-    /** Query id. */
-    private Long qryId;
-
-    /** Query type name. */
-    private String typeName;
-
-    /**
-     * @param sqlQry Sql query.
-     */
-    public void sqlQuery(String sqlQry) {
-        this.sqlQry = sqlQry;
-    }
-
-    /**
-     * @return Sql query.
-     */
-    public String sqlQuery() {
-        return sqlQry;
-    }
-
-    /**
-     * @param args Sql query arguments.
-     */
-    public void arguments(Object[] args) {
-        this.args = args;
-    }
-
-    /**
-     * @return Sql query arguments.
-     */
-    public Object[] arguments() {
-        return args;
-    }
-
-    /**
-     * @param pageSize Page size.
-     */
-    public void pageSize(Integer pageSize) {
-        this.pageSize = pageSize;
-    }
-
-    /**
-     * @return Page size.
-     */
-    public int pageSize() {
-        return pageSize;
-    }
-
-    /**
-     * @param cacheName Cache name.
-     */
-    public void cacheName(String cacheName) {
-        this.cacheName = cacheName;
-    }
-
-    /**
-     * @return Cache name.
-     */
-    public String cacheName() {
-        return cacheName;
-    }
-
-    /**
-     * @param id Query id.
-     */
-    public void queryId(Long id) {
-        this.qryId = id;
-    }
-
-    /**
-     * @return Query id.
-     */
-    public Long queryId() {
-        return qryId;
-    }
-
-    /**
-     * @param typeName Query type name.
-     */
-    public void typeName(String typeName) {
-        this.typeName = typeName;
-    }
-
-    /**
-     * @return Query type name.
-     */
-    public String typeName() {
-        return typeName;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/12235254/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/GridJettyJsonConfig.java
----------------------------------------------------------------------
diff --git a/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/GridJettyJsonConfig.java b/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/GridJettyJsonConfig.java
index 60b44ee..0adbc14 100644
--- a/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/GridJettyJsonConfig.java
+++ b/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/GridJettyJsonConfig.java
@@ -17,10 +17,19 @@
 
 package org.apache.ignite.internal.processors.rest.protocols.http.jetty;
 
+import java.text.DateFormat;
 import java.util.UUID;
+import net.sf.json.JSONObject;
 import net.sf.json.JsonConfig;
+import net.sf.json.processors.JsonBeanProcessor;
+import net.sf.json.processors.JsonBeanProcessorMatcher;
 import net.sf.json.processors.JsonValueProcessor;
 
+import java.util.*;
+import net.sf.json.processors.JsonValueProcessorMatcher;
+import org.apache.ignite.internal.processors.cache.query.GridCacheSqlIndexMetadata;
+import org.apache.ignite.internal.processors.cache.query.GridCacheSqlMetadata;
+
 /**
  * Jetty protocol json configuration.
  */
@@ -29,21 +38,158 @@ public class GridJettyJsonConfig extends JsonConfig {
      * Constructs default jetty json config.
      */
     public GridJettyJsonConfig() {
-        registerJsonValueProcessor(UUID.class, new ToStringJsonProcessor());
+        registerJsonValueProcessor(UUID.class, new UUIDToStringJsonProcessor());
+        registerJsonValueProcessor(Date.class, new DateToStringJsonProcessor());
+        registerJsonValueProcessor(java.sql.Date.class, new DateToStringJsonProcessor());
+
+        registerJsonBeanProcessor(GridCacheSqlMetadata.class, new GridCacheSqlMetadataBeanProcessor());
+        registerJsonValueProcessor(GridCacheSqlIndexMetadata.class, new GridCacheSqlIndexMetadataToJson());
+
+        setJsonBeanProcessorMatcher(new GridJettyJsonBeanProcessorMatcher());
+        setJsonValueProcessorMatcher(new GridJettyJsonValueProcessorMatcher());
     }
 
     /**
-     * Helper class for simple to-string conversion for the beans.
+     * Helper class for simple to-string conversion for {@link UUID}.
      */
-    private static class ToStringJsonProcessor implements JsonValueProcessor {
+    private static class UUIDToStringJsonProcessor implements JsonValueProcessor {
         /** {@inheritDoc} */
         @Override public Object processArrayValue(Object val, JsonConfig jsonCfg) {
-            throw new UnsupportedOperationException("Serialize array to string is not supported: " + val);
+            if (val == null)
+                return new JSONObject(true);
+
+            if (val instanceof UUID)
+                return val.toString();
+
+            throw new UnsupportedOperationException("Serialize value to json is not supported: " + val);
         }
 
         /** {@inheritDoc} */
         @Override public Object processObjectValue(String key, Object val, JsonConfig jsonCfg) {
-            return val == null ? null : val.toString();
+            return processArrayValue(val, jsonCfg);
+        }
+    }
+
+    /**
+     * Helper class for simple to-string conversion for {@link Date}.
+     */
+    private static class DateToStringJsonProcessor implements JsonValueProcessor {
+        private final DateFormat enUsFormat
+            =  DateFormat.getDateTimeInstance(DateFormat.DEFAULT, DateFormat.DEFAULT, Locale.US);
+
+        /** {@inheritDoc} */
+        @Override public synchronized Object processArrayValue(Object val, JsonConfig jsonCfg) {
+            if (val == null)
+                return new JSONObject(true);
+
+            if (val instanceof Date)
+                return enUsFormat.format(val);
+
+            throw new UnsupportedOperationException("Serialize value to json is not supported: " + val);
+        }
+
+        /** {@inheritDoc} */
+        @Override public synchronized Object processObjectValue(String key, Object val, JsonConfig jsonCfg) {
+            return processArrayValue(val, jsonCfg);
+        }
+    }
+
+    /**
+     * Helper class for simple to-json conversion for {@link GridCacheSqlMetadata}.
+     */
+    private static class GridCacheSqlMetadataBeanProcessor implements JsonBeanProcessor {
+        /** {@inheritDoc} */
+        @Override public JSONObject processBean(Object bean, JsonConfig jsonCfg) {
+            if (bean == null)
+                return new JSONObject(true);
+
+            if (bean instanceof GridCacheSqlMetadata) {
+                GridCacheSqlMetadata r = (GridCacheSqlMetadata) bean;
+
+                return new JSONObject()
+                    .element("cacheName", r.cacheName(), jsonCfg)
+                    .element("types", r.types(), jsonCfg)
+                    .element("keyClasses", r.keyClasses(), jsonCfg)
+                    .element("valClasses", r.valClasses(), jsonCfg)
+                    .element("fields", r.fields(), jsonCfg)
+                    .element("indexes", r.indexes(), jsonCfg);
+            }
+
+            throw new UnsupportedOperationException("Serialize bean to json is not supported: " + bean);
+        }
+    }
+
+    /**
+     * Helper class for simple to-json conversion for {@link GridCacheSqlIndexMetadata}.
+     */
+    private static class GridCacheSqlIndexMetadataToJson implements JsonValueProcessor {
+        /** {@inheritDoc} */
+        @Override public Object processArrayValue(Object val, JsonConfig jsonCfg) {
+            if (val == null)
+                return new JSONObject(true);
+
+            if (val instanceof GridCacheSqlIndexMetadata) {
+                GridCacheSqlIndexMetadata r = (GridCacheSqlIndexMetadata) val;
+
+                return new JSONObject()
+                    .element("name", r.name())
+                    .element("fields", r.fields())
+                    .element("descendings", r.descendings())
+                    .element("unique", r.unique());
+            }
+
+            throw new UnsupportedOperationException("Serialize array to string is not supported: " + val);
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object processObjectValue(String key, Object value, JsonConfig jsonCfg) {
+            return processArrayValue(value, jsonCfg);
+        }
+    }
+
+    /**
+     * Class for finding a matching JsonBeanProcessor. Matches the target class with instanceOf.
+     */
+    private static final class GridJettyJsonBeanProcessorMatcher extends JsonBeanProcessorMatcher {
+        /** {@inheritDoc} */
+        @Override public Object getMatch(Class target, Set keys) {
+            if (target == null || keys == null)
+                return null;
+
+            if (keys.contains(target))
+                return target;
+
+            for (Object key : keys) {
+                Class<?> clazz = (Class<?>) key;
+
+                if (clazz.isAssignableFrom(target))
+                    return key;
+            }
+
+            return null;
+        }
+    }
+
+    /**
+     * Class for finding a matching JsonValueProcessor. Matches the target class with instanceOf.
+     */
+    private static final class GridJettyJsonValueProcessorMatcher extends JsonValueProcessorMatcher {
+        /** {@inheritDoc} */
+        @Override public Object getMatch(Class target, Set keys) {
+            if (target == null || keys == null)
+                return null;
+
+            if (keys.contains(target))
+                return target;
+
+            for (Object key : keys) {
+                Class<?> clazz = (Class<?>) key;
+
+                if (clazz.isAssignableFrom(target))
+                    return key;
+            }
+
+            return null;
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/12235254/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 9b82098..48a60a9 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
@@ -51,7 +51,7 @@ import org.apache.ignite.internal.processors.rest.request.GridRestLogRequest;
 import org.apache.ignite.internal.processors.rest.request.GridRestRequest;
 import org.apache.ignite.internal.processors.rest.request.GridRestTaskRequest;
 import org.apache.ignite.internal.processors.rest.request.GridRestTopologyRequest;
-import org.apache.ignite.internal.processors.rest.request.RestSqlQueryRequest;
+import org.apache.ignite.internal.processors.rest.request.RestQueryRequest;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteClosure;
@@ -64,6 +64,7 @@ import static org.apache.ignite.internal.processors.rest.GridRestCommand.CACHE_C
 import static org.apache.ignite.internal.processors.rest.GridRestCommand.CACHE_GET_ALL;
 import static org.apache.ignite.internal.processors.rest.GridRestCommand.CACHE_PUT_ALL;
 import static org.apache.ignite.internal.processors.rest.GridRestCommand.CACHE_REMOVE_ALL;
+import static org.apache.ignite.internal.processors.rest.GridRestCommand.EXECUTE_SQL_QUERY;
 import static org.apache.ignite.internal.processors.rest.GridRestResponse.STATUS_FAILED;
 
 /**
@@ -84,19 +85,15 @@ public class GridJettyRestHandler extends AbstractHandler {
 
     /** Logger. */
     private final IgniteLogger log;
-
+    /** Authentication checker. */
+    private final IgniteClosure<String, Boolean> authChecker;
     /** Request handlers. */
     private GridRestProtocolHandler hnd;
-
     /** Default page. */
     private volatile String dfltPage;
-
     /** Favicon. */
     private volatile byte[] favicon;
 
-    /** Authentication checker. */
-    private final IgniteClosure<String, Boolean> authChecker;
-
     /**
      * Creates new HTTP requests handler.
      *
@@ -135,6 +132,74 @@ public class GridJettyRestHandler extends AbstractHandler {
     }
 
     /**
+     * Retrieves long value from parameters map.
+     *
+     * @param key Key.
+     * @param params Parameters map.
+     * @param dfltVal Default value.
+     * @return Long value from parameters map or {@code dfltVal} if null
+     *     or not exists.
+     * @throws IgniteCheckedException If parsing failed.
+     */
+    @Nullable private static Long longValue(String key, Map<String, Object> params, Long dfltVal) throws IgniteCheckedException {
+        assert key != null;
+
+        String val = (String) params.get(key);
+
+        try {
+            return val == null ? dfltVal : Long.valueOf(val);
+        }
+        catch (NumberFormatException ignore) {
+            throw new IgniteCheckedException("Failed to parse parameter of Long type [" + key + "=" + val + "]");
+        }
+    }
+
+    /**
+     * Retrieves int value from parameters map.
+     *
+     * @param key Key.
+     * @param params Parameters map.
+     * @param dfltVal Default value.
+     * @return Integer value from parameters map or {@code dfltVal} if null
+     *     or not exists.
+     * @throws IgniteCheckedException If parsing failed.
+     */
+    @Nullable private static Integer intValue(String key, Map<String, Object> params, Integer dfltVal) throws IgniteCheckedException {
+        assert key != null;
+
+        String val = (String) params.get(key);
+
+        try {
+            return val == null ? dfltVal : Integer.valueOf(val);
+        }
+        catch (NumberFormatException ignore) {
+            throw new IgniteCheckedException("Failed to parse parameter of Integer type [" + key + "=" + val + "]");
+        }
+    }
+
+    /**
+     * Retrieves UUID value from parameters map.
+     *
+     * @param key Key.
+     * @param params Parameters map.
+     * @return UUID value from parameters map or {@code null} if null
+     *     or not exists.
+     * @throws IgniteCheckedException If parsing failed.
+     */
+    @Nullable private static UUID uuidValue(String key, Map<String, Object> params) throws IgniteCheckedException {
+        assert key != null;
+
+        String val = (String) params.get(key);
+
+        try {
+            return val == null ? null : UUID.fromString(val);
+        }
+        catch (NumberFormatException ignore) {
+            throw new IgniteCheckedException("Failed to parse parameter of UUID type [" + key + "=" + val + "]");
+        }
+    }
+
+    /**
      * @throws IOException If failed.
      */
     private void initDefaultPage() throws IOException {
@@ -396,6 +461,7 @@ public class GridJettyRestHandler extends AbstractHandler {
             case CACHE_CAS:
             case CACHE_METRICS:
             case CACHE_SIZE:
+            case CACHE_METADATA:
             case CACHE_REPLACE:
             case CACHE_APPEND:
             case CACHE_PREPEND: {
@@ -497,9 +563,9 @@ public class GridJettyRestHandler extends AbstractHandler {
 
             case EXECUTE_SQL_QUERY:
             case EXECUTE_SQL_FIELDS_QUERY: {
-                RestSqlQueryRequest restReq0 = new RestSqlQueryRequest();
+                RestQueryRequest restReq0 = new RestQueryRequest();
 
-                restReq0.sqlQuery((String) params.get("qry"));
+                restReq0.sqlQuery((String)params.get("qry"));
 
                 restReq0.arguments(values("arg", params).toArray());
 
@@ -512,20 +578,46 @@ public class GridJettyRestHandler extends AbstractHandler {
 
                 restReq0.cacheName((String)params.get("cacheName"));
 
+                if (cmd == EXECUTE_SQL_QUERY)
+                    restReq0.queryType(RestQueryRequest.QueryType.SQL);
+                else
+                    restReq0.queryType(RestQueryRequest.QueryType.SQL_FIELDS);
+
+                restReq = restReq0;
+
+                break;
+            }
+
+            case EXECUTE_SCAN_QUERY: {
+                RestQueryRequest restReq0 = new RestQueryRequest();
+
+                restReq0.sqlQuery((String)params.get("qry"));
+
+                String pageSize = (String)params.get("pageSize");
+
+                if (pageSize != null)
+                    restReq0.pageSize(Integer.parseInt(pageSize));
+
+                restReq0.cacheName((String)params.get("cacheName"));
+
+                restReq0.className((String)params.get("classname"));
+
+                restReq0.queryType(RestQueryRequest.QueryType.SCAN);
+
                 restReq = restReq0;
 
                 break;
             }
 
             case FETCH_SQL_QUERY: {
-                RestSqlQueryRequest restReq0 = new RestSqlQueryRequest();
+                RestQueryRequest restReq0 = new RestQueryRequest();
 
                 String qryId = (String) params.get("qryId");
 
                 if (qryId != null)
                     restReq0.queryId(Long.parseLong(qryId));
 
-                String pageSize = (String) params.get("pageSize");
+                String pageSize = (String)params.get("pageSize");
 
                 if (pageSize != null)
                     restReq0.pageSize(Integer.parseInt(pageSize));
@@ -538,7 +630,7 @@ public class GridJettyRestHandler extends AbstractHandler {
             }
 
             case CLOSE_SQL_QUERY: {
-                RestSqlQueryRequest restReq0 = new RestSqlQueryRequest();
+                RestQueryRequest restReq0 = new RestQueryRequest();
 
                 String qryId = (String) params.get("qryId");
 
@@ -601,74 +693,6 @@ public class GridJettyRestHandler extends AbstractHandler {
     }
 
     /**
-     * Retrieves long value from parameters map.
-     *
-     * @param key Key.
-     * @param params Parameters map.
-     * @param dfltVal Default value.
-     * @return Long value from parameters map or {@code dfltVal} if null
-     *     or not exists.
-     * @throws IgniteCheckedException If parsing failed.
-     */
-    @Nullable private static Long longValue(String key, Map<String, Object> params, Long dfltVal) throws IgniteCheckedException {
-        assert key != null;
-
-        String val = (String) params.get(key);
-
-        try {
-            return val == null ? dfltVal : Long.valueOf(val);
-        }
-        catch (NumberFormatException ignore) {
-            throw new IgniteCheckedException("Failed to parse parameter of Long type [" + key + "=" + val + "]");
-        }
-    }
-
-    /**
-     * Retrieves int value from parameters map.
-     *
-     * @param key Key.
-     * @param params Parameters map.
-     * @param dfltVal Default value.
-     * @return Integer value from parameters map or {@code dfltVal} if null
-     *     or not exists.
-     * @throws IgniteCheckedException If parsing failed.
-     */
-    @Nullable private static Integer intValue(String key, Map<String, Object> params, Integer dfltVal) throws IgniteCheckedException {
-        assert key != null;
-
-        String val = (String) params.get(key);
-
-        try {
-            return val == null ? dfltVal : Integer.valueOf(val);
-        }
-        catch (NumberFormatException ignore) {
-            throw new IgniteCheckedException("Failed to parse parameter of Integer type [" + key + "=" + val + "]");
-        }
-    }
-
-    /**
-     * Retrieves UUID value from parameters map.
-     *
-     * @param key Key.
-     * @param params Parameters map.
-     * @return UUID value from parameters map or {@code null} if null
-     *     or not exists.
-     * @throws IgniteCheckedException If parsing failed.
-     */
-    @Nullable private static UUID uuidValue(String key, Map<String, Object> params) throws IgniteCheckedException {
-        assert key != null;
-
-        String val = (String) params.get(key);
-
-        try {
-            return val == null ? null : UUID.fromString(val);
-        }
-        catch (NumberFormatException ignore) {
-            throw new IgniteCheckedException("Failed to parse parameter of UUID type [" + key + "=" + val + "]");
-        }
-    }
-
-    /**
      * Gets values referenced by sequential keys, e.g. {@code key1...keyN}.
      *
      * @param keyPrefix Key prefix, e.g. {@code key} for {@code key1...keyN}.
@@ -736,4 +760,4 @@ public class GridJettyRestHandler extends AbstractHandler {
 
         return null;
     }
-}
\ No newline at end of file
+}