Repository: ignite
Updated Branches:
  refs/heads/ignite-3443 [created] 8ef730c88


IGNITE-3443 WIP


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

Branch: refs/heads/ignite-3443
Commit: 8ef730c882e735e797086de0002194fdb62f4446
Parents: 58d0544
Author: Alexey Kuznetsov <akuznet...@apache.org>
Authored: Tue Jul 26 16:36:15 2016 +0700
Committer: Alexey Kuznetsov <akuznet...@apache.org>
Committed: Tue Jul 26 16:36:15 2016 +0700

----------------------------------------------------------------------
 .../ignite/cache/query/QueryMetricsEx.java      |  35 +++++
 .../processors/cache/IgniteCacheProxy.java      |  11 +-
 .../processors/cache/query/CacheQueryType.java  |   7 +-
 .../cache/query/GridCacheQueryAdapter.java      |   6 +-
 .../query/GridCacheQueryFutureAdapter.java      |  17 ++-
 .../cache/query/GridCacheQueryManager.java      |  16 +-
 .../query/GridCacheQueryMetricsAdapter.java     |   7 +-
 .../processors/query/GridQueryProcessor.java    | 145 ++++++++++---------
 8 files changed, 161 insertions(+), 83 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/8ef730c8/modules/core/src/main/java/org/apache/ignite/cache/query/QueryMetricsEx.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/cache/query/QueryMetricsEx.java 
b/modules/core/src/main/java/org/apache/ignite/cache/query/QueryMetricsEx.java
new file mode 100644
index 0000000..085dea5
--- /dev/null
+++ 
b/modules/core/src/main/java/org/apache/ignite/cache/query/QueryMetricsEx.java
@@ -0,0 +1,35 @@
+/*
+ * 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.cache.query;
+
+import org.apache.ignite.internal.processors.cache.query.CacheQueryType;
+
+/**
+ * Per individual query metrics.
+ */
+public interface QueryMetricsEx extends QueryMetrics {
+    /**
+     * @return String representation of query.
+     */
+    public String query();
+
+    /**
+     * @return Query type.
+     */
+    public CacheQueryType queryType();
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8ef730c8/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
index 0005530..9e7d0e1 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
@@ -66,6 +66,7 @@ import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.query.CacheQuery;
 import org.apache.ignite.internal.processors.cache.query.CacheQueryFuture;
+import org.apache.ignite.internal.processors.cache.query.CacheQueryType;
 import org.apache.ignite.internal.processors.query.GridQueryProcessor;
 import org.apache.ignite.internal.util.GridCloseableIteratorAdapter;
 import org.apache.ignite.internal.util.GridEmptyIterator;
@@ -241,6 +242,7 @@ public class IgniteCacheProxy<K, V> extends 
AsyncSupportAdapter<IgniteCache<K, V
         }
     }
 
+    /** {@inheritDoc} */
     @Override public CacheMetrics localMetrics() {
         GridCacheGateway<K, V> gate = this.gate;
 
@@ -268,6 +270,7 @@ public class IgniteCacheProxy<K, V> extends 
AsyncSupportAdapter<IgniteCache<K, V
         }
     }
 
+    /** {@inheritDoc} */
     @Override public CacheMetricsMXBean localMxBean() {
         GridCacheGateway<K, V> gate = this.gate;
 
@@ -485,8 +488,8 @@ public class IgniteCacheProxy<K, V> extends 
AsyncSupportAdapter<IgniteCache<K, V
             if (grp != null)
                 qry.projection(grp);
 
-            final GridCloseableIterator<Entry<K, V>> iter = 
ctx.kernalContext().query().executeQuery(ctx,
-                new IgniteOutClosureX<GridCloseableIterator<Entry<K, V>>>() {
+            final GridCloseableIterator<Entry<K, V>> iter = 
ctx.kernalContext().query().executeQuery(CacheQueryType.SCAN,
+                ctx.name(), ctx, new 
IgniteOutClosureX<GridCloseableIterator<Entry<K, V>>>() {
                     @Override public GridCloseableIterator<Entry<K, V>> 
applyx() throws IgniteCheckedException {
                         final GridCloseableIterator<Map.Entry> iter0 = 
qry.executeScanQuery();
 
@@ -521,7 +524,7 @@ public class IgniteCacheProxy<K, V> extends 
AsyncSupportAdapter<IgniteCache<K, V
             if (grp != null)
                 qry.projection(grp);
 
-            fut = ctx.kernalContext().query().executeQuery(ctx,
+            fut = 
ctx.kernalContext().query().executeQuery(CacheQueryType.FULL_TEXT, p.getText(), 
ctx,
                 new IgniteOutClosureX<CacheQueryFuture<Map.Entry<K, V>>>() {
                     @Override public CacheQueryFuture<Map.Entry<K, V>> 
applyx() throws IgniteCheckedException {
                         return qry.execute();
@@ -534,7 +537,7 @@ public class IgniteCacheProxy<K, V> extends 
AsyncSupportAdapter<IgniteCache<K, V
             if (grp != null)
                 qry.projection(grp);
 
-            fut = ctx.kernalContext().query().executeQuery(ctx,
+            fut = ctx.kernalContext().query().executeQuery(CacheQueryType.SPI, 
"", ctx,
                 new IgniteOutClosureX<CacheQueryFuture<Map.Entry<K, V>>>() {
                     @Override public CacheQueryFuture<Map.Entry<K, V>> 
applyx() throws IgniteCheckedException {
                         return qry.execute(((SpiQuery)filter).getArgs());

http://git-wip-us.apache.org/repos/asf/ignite/blob/8ef730c8/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryType.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryType.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryType.java
index bb8bc33..8ff761b 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryType.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryType.java
@@ -43,5 +43,8 @@ public enum CacheQueryType {
     CONTINUOUS,
 
     /** SPI query. */
-    SPI
-}
\ No newline at end of file
+    SPI,
+
+    /** Cache set items query. */
+    SET
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8ef730c8/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
index 90e14f4..3414766 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
@@ -391,13 +391,15 @@ public class GridCacheQueryAdapter<T> implements 
CacheQuery<T> {
     }
 
     /**
+     * @param qryType Query type.
+     * @param qry Query description.
      * @param res Query result.
      * @param err Error or {@code null} if query executed successfully.
      * @param startTime Start time.
      * @param duration Duration.
      */
-    public void onCompleted(Object res, Throwable err, long startTime, long 
duration) {
-        GridQueryProcessor.onCompleted(cctx, res, err, startTime, duration, 
log);
+    public void onCompleted(CacheQueryType qryType, String qry, Object res, 
Throwable err, long startTime, long duration) {
+        GridQueryProcessor.onCompleted(qryType, qry, cctx, res, err, 
startTime, duration, log);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/8ef730c8/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java
index db519f5..b759a37 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.cache.query;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.EnumMap;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.LinkedList;
@@ -63,6 +64,18 @@ public abstract class GridCacheQueryFutureAdapter<K, V, R> 
extends GridFutureAda
     /** */
     private static final Object NULL = new Object();
 
+    /** */
+    private static final Map<GridCacheQueryType, CacheQueryType> QRY_TYPES = 
new EnumMap<>(GridCacheQueryType.class);
+
+    static {
+        QRY_TYPES.put(GridCacheQueryType.SPI, CacheQueryType.SPI);
+        QRY_TYPES.put(GridCacheQueryType.SCAN, CacheQueryType.SCAN);
+        QRY_TYPES.put(GridCacheQueryType.SQL, CacheQueryType.SQL);
+        QRY_TYPES.put(GridCacheQueryType.SQL_FIELDS, 
CacheQueryType.SQL_FIELDS);
+        QRY_TYPES.put(GridCacheQueryType.TEXT, CacheQueryType.FULL_TEXT);
+        QRY_TYPES.put(GridCacheQueryType.SET, CacheQueryType.SET);
+    }
+
     /** Cache context. */
     protected GridCacheContext<K, V> cctx;
 
@@ -155,7 +168,9 @@ public abstract class GridCacheQueryFutureAdapter<K, V, R> 
extends GridFutureAda
     @Override public boolean onDone(Collection<R> res, Throwable err) {
         cctx.time().removeTimeoutObject(this);
 
-        qry.query().onCompleted(res, err, startTime(), duration());
+        GridCacheQueryAdapter<?> qryAdapter = qry.query();
+
+        qry.query().onCompleted(QRY_TYPES.get(qryAdapter.type()), 
qryAdapter.clause(), res, err, startTime(), duration());
 
         return super.onDone(res, err);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/8ef730c8/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
index 60e81e2..c366877 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
@@ -1119,9 +1119,8 @@ public abstract class GridCacheQueryManager<K, V> extends 
GridCacheManagerAdapte
 
                     key = (K)cctx.unwrapBinaryIfNeeded(key, keepBinary);
 
-                    if (filter != null || locNode) {
+                    if (filter != null || locNode)
                         val = (V)cctx.unwrapBinaryIfNeeded(val, keepBinary);
-                    }
 
                     if (filter != null && !filter.apply(key, val))
                         continue;
@@ -1690,6 +1689,8 @@ public abstract class GridCacheQueryManager<K, V> extends 
GridCacheManagerAdapte
 
         long startTime = U.currentTimeMillis();
 
+        final String namex = cctx.namex();
+
         try {
             assert qry.type() == SCAN;
 
@@ -1698,7 +1699,6 @@ public abstract class GridCacheQueryManager<K, V> extends 
GridCacheManagerAdapte
 
             final String taskName = 
cctx.kernalContext().task().resolveTaskName(qry.taskHash());
             final IgniteBiPredicate filter = qry.scanFilter();
-            final String namex = cctx.namex();
             final ClusterNode locNode = cctx.localNode();
             final UUID subjId = qry.subjectId();
 
@@ -1723,7 +1723,7 @@ public abstract class GridCacheQueryManager<K, V> extends 
GridCacheManagerAdapte
             if (updStatisticsIfNeeded) {
                 needUpdStatistics = false;
 
-                cctx.queries().onCompleted(U.currentTimeMillis() - startTime, 
false);
+                cctx.queries().onCompleted(CacheQueryType.SCAN, namex, 
U.currentTimeMillis() - startTime, false);
             }
 
             final boolean readEvt = 
cctx.gridEvents().isRecordable(EVT_CACHE_QUERY_OBJECT_READ);
@@ -1776,7 +1776,7 @@ public abstract class GridCacheQueryManager<K, V> extends 
GridCacheManagerAdapte
         }
         catch (Exception e) {
             if (needUpdStatistics)
-                cctx.queries().onCompleted(U.currentTimeMillis() - startTime, 
true);
+                cctx.queries().onCompleted(CacheQueryType.SCAN, namex, 
U.currentTimeMillis() - startTime, true);
 
             throw e;
         }
@@ -2071,10 +2071,14 @@ public abstract class GridCacheQueryManager<K, V> 
extends GridCacheManagerAdapte
     }
 
     /**
+     * @param qryType Query type.
+     * @param qry Query description.
      * @param duration Execution duration.
      * @param fail {@code true} if execution failed.
      */
-    public void onCompleted(long duration, boolean fail) {
+    public void onCompleted(CacheQueryType qryType, String qry, long duration, 
boolean fail) {
+        log.warning("Query type: " + qryType + ", query: " + qry);
+
         metrics.onQueryCompleted(duration, fail);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/8ef730c8/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryMetricsAdapter.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryMetricsAdapter.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryMetricsAdapter.java
index 1928ea5..27996a6 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryMetricsAdapter.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryMetricsAdapter.java
@@ -21,6 +21,8 @@ import java.io.Externalizable;
 import java.io.IOException;
 import java.io.ObjectInput;
 import java.io.ObjectOutput;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
 import org.apache.ignite.cache.query.QueryMetrics;
 import org.apache.ignite.internal.util.GridAtomicLong;
 import org.apache.ignite.internal.util.typedef.internal.S;
@@ -56,6 +58,9 @@ public class GridCacheQueryMetricsAdapter implements 
QueryMetrics, Externalizabl
     /** Number of fails. */
     private final LongAdder8 fails = new LongAdder8();
 
+    /** TODO IGNITE-3443 */
+    private final Map<String, String> perQryMetrics = new 
ConcurrentHashMap<>(100);
+
     /** {@inheritDoc} */
     @Override public long minimumTime() {
         return minTime.get();
@@ -170,4 +175,4 @@ public class GridCacheQueryMetricsAdapter implements 
QueryMetrics, Externalizabl
     @Override public String toString() {
         return S.toString(GridCacheQueryMetricsAdapter.class, this);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8ef730c8/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index 0dd542e..dfbd516 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@ -17,6 +17,30 @@
 
 package org.apache.ignite.internal.processors.query;
 
+import java.lang.reflect.AccessibleObject;
+import java.lang.reflect.Field;
+import java.lang.reflect.Member;
+import java.lang.reflect.Method;
+import java.math.BigDecimal;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutorService;
+import javax.cache.Cache;
+import javax.cache.CacheException;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
@@ -68,31 +92,6 @@ import org.apache.ignite.spi.indexing.IndexingQueryFilter;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
 
-import javax.cache.Cache;
-import javax.cache.CacheException;
-import java.lang.reflect.AccessibleObject;
-import java.lang.reflect.Field;
-import java.lang.reflect.Member;
-import java.lang.reflect.Method;
-import java.math.BigDecimal;
-import java.sql.Time;
-import java.sql.Timestamp;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.TreeSet;
-import java.util.UUID;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ExecutorService;
-
 import static org.apache.ignite.events.EventType.EVT_CACHE_QUERY_EXECUTED;
 import static org.apache.ignite.internal.IgniteComponentType.INDEXING;
 import static 
org.apache.ignite.internal.processors.query.GridQueryIndexType.FULLTEXT;
@@ -727,7 +726,7 @@ public class GridQueryProcessor extends 
GridProcessorAdapter {
             throw new IllegalStateException("Failed to execute query (grid is 
stopping).");
 
         try {
-            return executeQuery(cctx, new 
IgniteOutClosureX<QueryCursor<List<?>>>() {
+            return executeQuery(CacheQueryType.SQL_FIELDS, qry.getSql(), cctx, 
new IgniteOutClosureX<QueryCursor<List<?>>>() {
                 @Override public QueryCursor<List<?>> applyx() throws 
IgniteCheckedException {
                     return idx.queryTwoStep(cctx, qry);
                 }
@@ -753,7 +752,7 @@ public class GridQueryProcessor extends 
GridProcessorAdapter {
             throw new IllegalStateException("Failed to execute query (grid is 
stopping).");
 
         try {
-            return executeQuery(cctx, new 
IgniteOutClosureX<QueryCursor<Cache.Entry<K, V>>>() {
+            return executeQuery(CacheQueryType.SQL, qry.getSql(), cctx, new 
IgniteOutClosureX<QueryCursor<Cache.Entry<K, V>>>() {
                 @Override public QueryCursor<Cache.Entry<K, V>> applyx() 
throws IgniteCheckedException {
                     return idx.queryTwoStep(cctx, qry);
                 }
@@ -783,6 +782,8 @@ public class GridQueryProcessor extends 
GridProcessorAdapter {
 
         try {
             return executeQuery(
+                CacheQueryType.SQL,
+                qry.getSql(),
                 cctx,
                 new IgniteOutClosureX<Iterator<Cache.Entry<K, V>>>() {
                     @Override public Iterator<Cache.Entry<K, V>> applyx() 
throws IgniteCheckedException {
@@ -883,28 +884,29 @@ public class GridQueryProcessor extends 
GridProcessorAdapter {
         try {
             final boolean keepBinary = cctx.keepBinary();
 
-            return executeQuery(cctx, new 
IgniteOutClosureX<QueryCursor<List<?>>>() {
-                @Override public QueryCursor<List<?>> applyx() throws 
IgniteCheckedException {
-                    String space = cctx.name();
-                    String sql = qry.getSql();
-                    Object[] args = qry.getArgs();
+            return executeQuery(CacheQueryType.SQL_FIELDS, qry.getSql(), cctx,
+                new IgniteOutClosureX<QueryCursor<List<?>>>() {
+                    @Override public QueryCursor<List<?>> applyx() throws 
IgniteCheckedException {
+                        String space = cctx.name();
+                        String sql = qry.getSql();
+                        Object[] args = qry.getArgs();
 
-                    final GridQueryFieldsResult res = 
idx.queryLocalSqlFields(space, sql, F.asList(args),
-                        idx.backupFilter(null, null), 
qry.isEnforceJoinOrder());
+                        final GridQueryFieldsResult res = 
idx.queryLocalSqlFields(space, sql, F.asList(args),
+                            idx.backupFilter(null, null), 
qry.isEnforceJoinOrder());
 
-                    sendQueryExecutedEvent(sql, args);
+                        sendQueryExecutedEvent(sql, args);
 
-                    QueryCursorImpl<List<?>> cursor = new 
QueryCursorImpl<>(new Iterable<List<?>>() {
-                        @Override public Iterator<List<?>> iterator() {
-                            return new 
GridQueryCacheObjectsIterator(res.iterator(), cctx, keepBinary);
-                        }
-                    });
+                        QueryCursorImpl<List<?>> cursor = new 
QueryCursorImpl<>(new Iterable<List<?>>() {
+                            @Override public Iterator<List<?>> iterator() {
+                                return new 
GridQueryCacheObjectsIterator(res.iterator(), cctx, keepBinary);
+                            }
+                        });
 
-                    cursor.fieldsMeta(res.metaData());
+                        cursor.fieldsMeta(res.metaData());
 
-                    return cursor;
-                }
-            }, true);
+                        return cursor;
+                    }
+                }, true);
         }
         catch (IgniteCheckedException e) {
             throw new CacheException(e);
@@ -999,14 +1001,13 @@ public class GridQueryProcessor extends 
GridProcessorAdapter {
      * @return Type name.
      */
     public static String typeName(String clsName) {
-        int packageEnd = clsName.lastIndexOf('.');
+        int pkgEnd = clsName.lastIndexOf('.');
 
-        if (packageEnd >= 0 && packageEnd < clsName.length() - 1)
-            clsName = clsName.substring(packageEnd + 1);
+        if (pkgEnd >= 0 && pkgEnd < clsName.length() - 1)
+            clsName = clsName.substring(pkgEnd + 1);
 
-        if (clsName.endsWith("[]")) {
+        if (clsName.endsWith("[]"))
             clsName = clsName.substring(0, clsName.length() - 2) + "_array";
-        }
 
         int parentEnd = clsName.lastIndexOf('$');
 
@@ -1037,20 +1038,21 @@ public class GridQueryProcessor extends 
GridProcessorAdapter {
         try {
             final GridCacheContext<?, ?> cctx = 
ctx.cache().internalCache(space).context();
 
-            return executeQuery(cctx, new 
IgniteOutClosureX<GridCloseableIterator<IgniteBiTuple<K, V>>>() {
-                @Override public GridCloseableIterator<IgniteBiTuple<K, V>> 
applyx() throws IgniteCheckedException {
-                    TypeDescriptor type = typesByName.get(new TypeName(space, 
resType));
+            return executeQuery(CacheQueryType.FULL_TEXT, clause, cctx,
+                new IgniteOutClosureX<GridCloseableIterator<IgniteBiTuple<K, 
V>>>() {
+                    @Override public GridCloseableIterator<IgniteBiTuple<K, 
V>> applyx() throws IgniteCheckedException {
+                        TypeDescriptor type = typesByName.get(new 
TypeName(space, resType));
 
-                    if (type == null || !type.registered())
-                        throw new CacheException("Failed to find SQL table for 
type: " + resType);
+                        if (type == null || !type.registered())
+                            throw new CacheException("Failed to find SQL table 
for type: " + resType);
 
-                    return idx.queryLocalText(
-                        space,
-                        clause,
-                        type,
-                        filters);
-                }
-            }, false);
+                        return idx.queryLocalText(
+                            space,
+                            clause,
+                            type,
+                            filters);
+                    }
+                }, false);
         }
         finally {
             busyLock.leaveBusy();
@@ -1650,11 +1652,13 @@ public class GridQueryProcessor extends 
GridProcessorAdapter {
     }
 
     /**
+     * @param qryType Query type.
+     * @param qry Query description.
      * @param cctx Cache context.
      * @param clo Closure.
      * @param complete Complete.
      */
-    public <R> R executeQuery(GridCacheContext<?, ?> cctx, 
IgniteOutClosureX<R> clo, boolean complete)
+    public <R> R executeQuery(CacheQueryType qryType, String qry, 
GridCacheContext<?, ?> cctx, IgniteOutClosureX<R> clo, boolean complete)
         throws IgniteCheckedException {
         final long startTime = U.currentTimeMillis();
 
@@ -1692,23 +1696,25 @@ public class GridQueryProcessor extends 
GridProcessorAdapter {
             cctx.queries().onExecuted(err != null);
 
             if (complete && err == null)
-                onCompleted(cctx, res, null, startTime, U.currentTimeMillis() 
- startTime, log);
+                onCompleted(qryType, qry, cctx, res, null, startTime, 
U.currentTimeMillis() - startTime, log);
         }
     }
 
     /**
-     * @param cctx Cctx.
+     * @param qryType Query type.
+     * @param qry Query description.
+     * @param cctx Cache context.
      * @param res Result.
      * @param err Err.
      * @param startTime Start time.
      * @param duration Duration.
      * @param log Logger.
      */
-    public static void onCompleted(GridCacheContext<?, ?> cctx, Object res, 
Throwable err,
+    public static void onCompleted(CacheQueryType qryType, String qry, 
GridCacheContext<?, ?> cctx, Object res, Throwable err,
         long startTime, long duration, IgniteLogger log) {
         boolean fail = err != null;
 
-        cctx.queries().onCompleted(duration, fail);
+        cctx.queries().onCompleted(qryType, qry, duration, fail);
 
         if (log.isTraceEnabled())
             log.trace("Query execution completed [startTime=" + startTime +
@@ -2377,6 +2383,11 @@ public class GridQueryProcessor extends 
GridProcessorAdapter {
      * The way to index.
      */
     private enum IndexType {
-        ASC, DESC, TEXT
+        /** Ascending index. */
+        ASC,
+        /** Descending index. */
+        DESC,
+        /** Text index. */
+        TEXT
     }
 }

Reply via email to