You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ak...@apache.org on 2016/08/03 11:29:30 UTC

[48/48] ignite git commit: IGNITE-3443 WIP on total metrics and per query metrics.

IGNITE-3443 WIP on total metrics and per query metrics.


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

Branch: refs/heads/ignite-3443
Commit: 8033f119ef4bc42dfa2a2657f852b45a5aab7c85
Parents: 2ec5d99
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Wed Aug 3 18:28:58 2016 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Wed Aug 3 18:28:58 2016 +0700

----------------------------------------------------------------------
 .../ignite/cache/query/QueryDetailsMetrics.java |  70 ++++++
 .../apache/ignite/cache/query/QueryMetrics.java |   8 +-
 .../ignite/cache/query/QueryMetricsEx.java      |  35 ---
 .../processors/cache/query/CacheQueryType.java  |  17 +-
 .../query/GridCacheQueryBaseMetricsAdapter.java | 188 ++++++++++++++++
 .../GridCacheQueryDetailsMetricsAdapter.java    |  84 ++++++++
 .../cache/query/GridCacheQueryManager.java      |  35 +--
 .../query/GridCacheQueryMetricsAdapter.java     | 213 ++++++++++---------
 .../internal/visor/cache/VisorCacheMetrics.java |   2 +-
 .../cache/VisorCacheQueryDetailsMetrics.java    | 108 ++++++++++
 .../visor/cache/VisorCacheQueryMetrics.java     |  18 +-
 .../visor/cache/VisorCacheQueryMetricsV2.java   |  45 ++++
 12 files changed, 637 insertions(+), 186 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/8033f119/modules/core/src/main/java/org/apache/ignite/cache/query/QueryDetailsMetrics.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/query/QueryDetailsMetrics.java b/modules/core/src/main/java/org/apache/ignite/cache/query/QueryDetailsMetrics.java
new file mode 100644
index 0000000..fd2d4fa
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/cache/query/QueryDetailsMetrics.java
@@ -0,0 +1,70 @@
+/*
+ * 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 aggregated by query type and Textual representation.
+ */
+public interface QueryDetailsMetrics {
+    /**
+     * @return Query type.
+     */
+    public CacheQueryType queryType();
+
+    /**
+     * @return Textual representation of query.
+     */
+    public String query();
+
+    /**
+     * Gets minimum execution time of query.
+     *
+     * @return Minimum execution time of query.
+     */
+    public long minimumTime();
+
+    /**
+     * Gets maximum execution time of query.
+     *
+     * @return Maximum execution time of query.
+     */
+    public long maximumTime();
+
+    /**
+     * Gets average execution time of query.
+     *
+     * @return Average execution time of query.
+     */
+    public double averageTime();
+
+    /**
+     * Gets total number execution of query.
+     *
+     * @return Number of executions.
+     */
+    public int executions();
+
+    /**
+     * Gets total number of times a query execution failed.
+     *
+     * @return Total number of times a query execution failed.
+     */
+    public int fails();
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8033f119/modules/core/src/main/java/org/apache/ignite/cache/query/QueryMetrics.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/query/QueryMetrics.java b/modules/core/src/main/java/org/apache/ignite/cache/query/QueryMetrics.java
index e376561..4bb0caa 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/query/QueryMetrics.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/query/QueryMetrics.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.cache.query;
 
+import java.util.List;
 import org.apache.ignite.internal.processors.cache.query.CacheQuery;
 
 /**
@@ -59,4 +60,9 @@ public interface QueryMetrics {
      * @return Total number of times a query execution failed.
      */
     public int fails();
-}
\ No newline at end of file
+
+    /**
+     * List of query metrics aggregated by query type and textual representation.
+     */
+    public List<QueryDetailsMetrics> details();
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8033f119/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
deleted file mode 100644
index 085dea5..0000000
--- a/modules/core/src/main/java/org/apache/ignite/cache/query/QueryMetricsEx.java
+++ /dev/null
@@ -1,35 +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.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/8033f119/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 8ff761b..0ebe8af 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
@@ -17,6 +17,8 @@
 
 package org.apache.ignite.internal.processors.cache.query;
 
+import org.jetbrains.annotations.Nullable;
+
 /**
  * Cache query type.
  * <p>
@@ -46,5 +48,18 @@ public enum CacheQueryType {
     SPI,
 
     /** Cache set items query. */
-    SET
+    SET;
+
+    /** Enumerated values. */
+    private static final CacheQueryType[] VALS = values();
+
+    /**
+     * Efficiently gets enumerated value from its ordinal.
+     *
+     * @param ord Ordinal value.
+     * @return Enumerated value.
+     */
+    @Nullable public static CacheQueryType fromOrdinal(byte ord) {
+        return ord >= 0 && ord < VALS.length ? VALS[ord] : null;
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/8033f119/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryBaseMetricsAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryBaseMetricsAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryBaseMetricsAdapter.java
new file mode 100644
index 0000000..e75b43c
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryBaseMetricsAdapter.java
@@ -0,0 +1,188 @@
+/*
+ * 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.cache.query;
+
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import org.apache.ignite.cache.query.QueryMetrics;
+import org.apache.ignite.internal.util.GridAtomicLong;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.jsr166.LongAdder8;
+
+/**
+ * Adapter for {@link QueryMetrics}.
+ */
+public abstract class GridCacheQueryBaseMetricsAdapter implements Externalizable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Minimum time of execution. */
+    private final GridAtomicLong minTime = new GridAtomicLong();
+
+    /** Maximum time of execution. */
+    private final GridAtomicLong maxTime = new GridAtomicLong();
+
+    /** Sum of execution time for all completed queries. */
+    private final LongAdder8 sumTime = new LongAdder8();
+
+    /**
+     * Average time of execution. If doesn't equal zero then this metrics set is copy from remote node and doesn't
+     * actually update.
+     */
+    private double avgTime;
+
+    /** Number of executions. */
+    private final LongAdder8 execs = new LongAdder8();
+
+    /** Number of completed executions. */
+    private final LongAdder8 completed = new LongAdder8();
+
+    /** Number of fails. */
+    private final LongAdder8 fails = new LongAdder8();
+
+    /**
+     * Gets minimum execution time of query.
+     *
+     * @return Minimum execution time of query.
+     */
+    public long minimumTime() {
+        return minTime.get();
+    }
+
+    /**
+     * Gets maximum execution time of query.
+     *
+     * @return Maximum execution time of query.
+     */
+    public long maximumTime() {
+        return maxTime.get();
+    }
+
+    /**
+     * Gets average execution time of query.
+     *
+     * @return Average execution time of query.
+     */
+    public double averageTime() {
+        if (avgTime > 0)
+            return avgTime;
+        else {
+            long val = completed.sum();
+
+            return val > 0 ? sumTime.sum() / val : 0;
+        }
+    }
+
+    /**
+     * Gets total number execution of query.
+     *
+     * @return Number of executions.
+     */
+    public int executions() {
+        return execs.intValue();
+    }
+
+    /**
+     * Gets total number of completed executions of query. This value is actual only for local node.
+     *
+     * @return Number of completed executions.
+     */
+    public int completedExecutions() {
+        return completed.intValue();
+    }
+
+    /**
+     * Gets total number of times a query execution failed.
+     *
+     * @return Total number of times a query execution failed.
+     */
+    public int fails() {
+        return fails.intValue();
+    }
+
+    /**
+     * Callback for query execution.
+     *
+     * @param fail {@code True} query executed unsuccessfully {@code false} otherwise.
+     */
+    public void onQueryExecute(boolean fail) {
+        execs.increment();
+
+        if (fail)
+            fails.increment();
+    }
+
+    /**
+     * Callback for completion of query execution.
+     *
+     * @param duration Duration of queue execution.
+     * @param fail {@code True} query executed unsuccessfully {@code false} otherwise.
+     */
+    protected void onQueryCompleted(long duration, boolean fail) {
+        minTime.setIfLess(duration);
+        maxTime.setIfGreater(duration);
+
+        if (fail)
+            fails.increment();
+        else {
+            completed.increment();
+
+            sumTime.add(duration);
+        }
+    }
+
+    /**
+     * Copy metrics to specified destination.
+     *
+     * @param m Metrics to fill.
+     */
+    protected void copy(GridCacheQueryBaseMetricsAdapter m) {
+        m.fails.add(fails.sum());
+        m.minTime.set(minTime.get());
+        m.maxTime.set(maxTime.get());
+        m.execs.add(execs.sum());
+        m.completed.add(completed.sum());
+        m.sumTime.add(sumTime.sum());
+        m.avgTime = avgTime;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeLong(minTime.get());
+        out.writeLong(maxTime.get());
+        out.writeDouble(averageTime());
+        out.writeInt(execs.intValue());
+        out.writeInt(fails.intValue());
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        minTime.set(in.readLong());
+        maxTime.set(in.readLong());
+        avgTime = in.readDouble();
+        execs.add(in.readInt());
+        fails.add(in.readInt());
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(GridCacheQueryBaseMetricsAdapter.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8033f119/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryDetailsMetricsAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryDetailsMetricsAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryDetailsMetricsAdapter.java
new file mode 100644
index 0000000..6af75f8
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryDetailsMetricsAdapter.java
@@ -0,0 +1,84 @@
+/*
+ * 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.cache.query;
+
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import org.apache.ignite.cache.query.QueryDetailsMetrics;
+import org.apache.ignite.cache.query.QueryMetrics;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * Adapter for {@link QueryMetrics}.
+ */
+public class GridCacheQueryDetailsMetricsAdapter extends GridCacheQueryBaseMetricsAdapter implements QueryDetailsMetrics {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Query type to track metrics. */
+    private CacheQueryType qryType;
+
+    /** Query text representation. */
+    private String qry;
+
+    /** {@inheritDoc} */
+    @Override  public CacheQueryType queryType() {
+        return qryType;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String query() {
+        return qry;
+    }
+
+    /**
+     * Merge with given metrics.
+     *
+     * @return Copy.
+     */
+    public GridCacheQueryDetailsMetricsAdapter copy() {
+        GridCacheQueryDetailsMetricsAdapter m = new GridCacheQueryDetailsMetricsAdapter();
+
+        copy(m);
+
+        return m;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        super.writeExternal(out);
+
+        U.writeEnum(out, qryType);
+        U.writeString(out, qry);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        super.readExternal(in);
+
+        qryType = CacheQueryType.fromOrdinal(in.readByte());
+        qry = U.readString(in);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(GridCacheQueryDetailsMetricsAdapter.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8033f119/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 4073652..d9973a5 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
@@ -38,9 +38,7 @@ import java.util.Queue;
 import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.Callable;
-import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.atomic.AtomicInteger;
 import javax.cache.Cache;
 import javax.cache.expiry.ExpiryPolicy;
 import org.apache.ignite.Ignite;
@@ -152,12 +150,6 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
     private volatile GridCacheQueryMetricsAdapter metrics = new GridCacheQueryMetricsAdapter();
 
     /** */
-    private volatile ConcurrentHashMap<String, String> latestQueryMetrics = new ConcurrentHashMap<>();
-
-    /** */
-    private volatile ConcurrentHashMap<String, AtomicInteger> aggregatedQueryMetrics = new ConcurrentHashMap<>();
-
-    /** */
     private final ConcurrentMap<UUID, RequestFutureMap> qryIters =
         new ConcurrentHashMap8<>();
 
@@ -2069,8 +2061,6 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
      */
     public void resetMetrics() {
         metrics = new GridCacheQueryMetricsAdapter();
-        latestQueryMetrics = new ConcurrentHashMap<>();
-        aggregatedQueryMetrics = new ConcurrentHashMap<>();
     }
 
     /**
@@ -2087,30 +2077,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
      * @param fail {@code true} if execution failed.
      */
     public void onCompleted(CacheQueryType qryType, String qry, long duration, boolean fail) {
-        metrics.onQueryCompleted(duration, fail);
-
-        latestQueryMetrics.put(qry, "Query [type=" + qryType + ", duration=" + duration + ", fail=" + fail + "]");
-
-        if (!aggregatedQueryMetrics.containsKey(qry))
-            aggregatedQueryMetrics.putIfAbsent(qry, new AtomicInteger());
-
-        aggregatedQueryMetrics.get(qry).incrementAndGet();
-
-        log.warning(">>> ---------------------------- <<<");
-
-        log.warning("latestQueryMetrics");
-
-        for (Map.Entry<String, String> entry : latestQueryMetrics.entrySet())
-            log.warning("  Query: " + entry.getKey() + " -> " + entry.getValue());
-
-        log.warning(">>> ============================ <<<");
-
-        log.warning("aggregatedQueryMetrics");
-
-        for (Map.Entry<String, AtomicInteger> entry : aggregatedQueryMetrics.entrySet())
-            log.warning("  Query: " + entry.getKey() + " -> " + entry.getValue());
-
-        log.warning(">>> ---------------------------- <<<");
+        metrics.onQueryCompleted(qryType, qry, duration, fail);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/8033f119/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 27996a6..eaf65be 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,116 +21,51 @@ 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 java.util.ArrayList;
+import java.util.List;
+import org.apache.ignite.cache.query.QueryDetailsMetrics;
 import org.apache.ignite.cache.query.QueryMetrics;
-import org.apache.ignite.internal.util.GridAtomicLong;
 import org.apache.ignite.internal.util.typedef.internal.S;
-import org.jsr166.LongAdder8;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jsr166.ConcurrentLinkedHashMap;
 
 /**
  * Adapter for {@link QueryMetrics}.
  */
-public class GridCacheQueryMetricsAdapter implements QueryMetrics, Externalizable {
+public class GridCacheQueryMetricsAdapter extends GridCacheQueryBaseMetricsAdapter implements QueryMetrics {
     /** */
     private static final long serialVersionUID = 0L;
 
-    /** Minimum time of execution. */
-    private final GridAtomicLong minTime = new GridAtomicLong();
-
-    /** Maximum time of execution. */
-    private final GridAtomicLong maxTime = new GridAtomicLong();
-
-    /** Sum of execution time for all completed queries. */
-    private final LongAdder8 sumTime = new LongAdder8();
-
-    /** Average time of execution.
-     * If doesn't equal zero then this metrics set is copy from remote node and doesn't actually update.
-     */
-    private double avgTime;
-
-    /** Number of executions. */
-    private final LongAdder8 execs = new LongAdder8();
-
-    /** Number of completed executions. */
-    private final LongAdder8 completed = new LongAdder8();
-
-    /** 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();
-    }
-
-    /** {@inheritDoc} */
-    @Override public long maximumTime() {
-        return maxTime.get();
-    }
-
-    /** {@inheritDoc} */
-    @Override public double averageTime() {
-        if (avgTime > 0)
-            return avgTime;
-        else {
-            long val = completed.sum();
-
-            return val > 0 ? sumTime.sum() / val : 0;
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public int executions() {
-        return execs.intValue();
-    }
+    private static final int MAX_CAP = 5;
 
-    /**
-     * Gets total number of completed executions of query.
-     * This value is actual only for local node.
-     *
-     * @return Number of completed executions.
-     */
-    public int completedExecutions() {
-        return completed.intValue();
-    }
+    /** TODO IGNITE-3443 */
+    private final ConcurrentLinkedHashMap<QueryMetricsKey, GridCacheQueryDetailsMetricsAdapter> details = new ConcurrentLinkedHashMap<>(MAX_CAP, 0.75f, 64, MAX_CAP);
 
     /** {@inheritDoc} */
-    @Override public int fails() {
-        return fails.intValue();
-    }
-
-    /**
-     * Callback for query execution.
-     *
-     * @param fail {@code True} query executed unsuccessfully {@code false} otherwise.
-     */
-    public void onQueryExecute(boolean fail) {
-        execs.increment();
-
-        if (fail)
-            fails.increment();
+    @Override public List<QueryDetailsMetrics> details() {
+        return new ArrayList<QueryDetailsMetrics>(details.values());
     }
 
     /**
      * Callback for completion of query execution.
      *
+     * @param qryType Query type.
+     * @param qry Query description.
      * @param duration Duration of queue execution.
      * @param fail {@code True} query executed unsuccessfully {@code false} otherwise.
      */
-    public void onQueryCompleted(long duration, boolean fail) {
-        minTime.setIfLess(duration);
-        maxTime.setIfGreater(duration);
+    public void onQueryCompleted(CacheQueryType qryType, String qry, long duration, boolean fail) {
+        onQueryCompleted(duration, fail);
 
-        if (fail)
-            fails.increment();
-        else {
-            completed.increment();
+        QueryMetricsKey key = new QueryMetricsKey(qryType, qry);
 
-            sumTime.add(duration);
-        }
+        if (!details.contains(key))
+            details.putIfAbsent(key, new GridCacheQueryDetailsMetricsAdapter());
+
+        GridCacheQueryDetailsMetricsAdapter dm = details.get(key);
+
+        dm.onQueryCompleted(duration, fail);
     }
 
     /**
@@ -142,37 +77,107 @@ public class GridCacheQueryMetricsAdapter implements QueryMetrics, Externalizabl
         GridCacheQueryMetricsAdapter m = new GridCacheQueryMetricsAdapter();
 
         // Not synchronized because accuracy isn't critical.
-        m.fails.add(fails.sum());
-        m.minTime.set(minTime.get());
-        m.maxTime.set(maxTime.get());
-        m.execs.add(execs.sum());
-        m.completed.add(completed.sum());
-        m.sumTime.add(sumTime.sum());
-        m.avgTime = avgTime;
+        copy(m);
+
+        m.details.putAll(details);
 
         return m;
     }
 
     /** {@inheritDoc} */
     @Override public void writeExternal(ObjectOutput out) throws IOException {
-        out.writeLong(minTime.get());
-        out.writeLong(maxTime.get());
-        out.writeDouble(averageTime());
-        out.writeInt(execs.intValue());
-        out.writeInt(fails.intValue());
+        super.writeExternal(out);
+
+        U.writeMap(out, details);
     }
 
     /** {@inheritDoc} */
     @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-        minTime.set(in.readLong());
-        maxTime.set(in.readLong());
-        avgTime = in.readDouble();
-        execs.add(in.readInt());
-        fails.add(in.readInt());
+        super.readExternal(in);
+
+        details.putAll(U.<QueryMetricsKey, GridCacheQueryDetailsMetricsAdapter>readMap(in));
     }
 
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(GridCacheQueryMetricsAdapter.class, this);
     }
+
+    /**
+     * Key for query metrics to store in map.
+     */
+    private static class QueryMetricsKey implements Externalizable {
+        /** Query type. */
+        private CacheQueryType qryType;
+
+        /** Query text descriptor: SQL, cache name, search text, ... */
+        private String qry;
+
+
+        /**
+         * Required by {@link Externalizable}.
+         */
+        public QueryMetricsKey() {
+            // No-op.
+        }
+
+        /**
+         * Constructor.
+         *
+         * @param qryType Query type.
+         * @param qry Query text descriptor.
+         */
+        public QueryMetricsKey(CacheQueryType qryType, String qry) {
+            this.qryType = qryType;
+            this.qry = qry;
+        }
+
+        /**
+         * @return Query type.
+         */
+        public CacheQueryType queryType() {
+            return qryType;
+        }
+
+        /**
+         * @return Query text descriptor: SQL, cache name, search text, ...
+         */
+        public String query() {
+            return qry;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void writeExternal(ObjectOutput out) throws IOException {
+            U.writeEnum(out, qryType);
+            U.writeString(out, qry);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+            qryType = CacheQueryType.fromOrdinal(in.readByte());
+            qry = U.readString(in);
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+
+            if (o == null || getClass() != o.getClass())
+                return false;
+
+            QueryMetricsKey key = (QueryMetricsKey)o;
+
+            return qryType == key.qryType && qry.equals(key.qry);
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            int res = qryType.hashCode();
+
+            res = 31 * res + qry.hashCode();
+
+            return res;
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/8033f119/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheMetrics.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheMetrics.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheMetrics.java
index 1204cbc..58d36ac 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheMetrics.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheMetrics.java
@@ -210,7 +210,7 @@ public class VisorCacheMetrics implements Serializable, LessNamingBean {
         commitsPerSec = perSecond(m.getAverageTxCommitTime());
         rollbacksPerSec = perSecond(m.getAverageTxRollbackTime());
 
-        qryMetrics = VisorCacheQueryMetrics.from(c.queryMetrics());
+        qryMetrics = new VisorCacheQueryMetrics().from(c.queryMetrics());
 
         dhtEvictQueueCurrSize = m.getDhtEvictQueueCurrentSize();
         txThreadMapSize = m.getTxThreadMapSize();

http://git-wip-us.apache.org/repos/asf/ignite/blob/8033f119/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheQueryDetailsMetrics.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheQueryDetailsMetrics.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheQueryDetailsMetrics.java
new file mode 100644
index 0000000..7fcbc24
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheQueryDetailsMetrics.java
@@ -0,0 +1,108 @@
+/*
+ * 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.visor.cache;
+
+import java.io.Serializable;
+import org.apache.ignite.cache.query.QueryDetailsMetrics;
+import org.apache.ignite.cache.query.QueryMetrics;
+import org.apache.ignite.internal.LessNamingBean;
+import org.apache.ignite.internal.processors.cache.query.CacheQueryType;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * Data transfer object for cache query metrics.
+ */
+public class VisorCacheQueryDetailsMetrics implements Serializable, LessNamingBean {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Query type. */
+    private CacheQueryType qryType;
+
+    /** Textual representation of query. */
+    private String qry;
+
+    /** Minimum execution time of query. */
+    private long minTime;
+
+    /** Maximum execution time of query. */
+    private long maxTime;
+
+    /** Average execution time of query. */
+    private double avgTime;
+
+    /** Number of executions. */
+    private int execs;
+
+    /** Total number of times a query execution failed. */
+    private int fails;
+
+    /**
+     * @param m Cache query metrics.
+     * @return Data transfer object for given cache metrics.
+     */
+    public VisorCacheQueryDetailsMetrics from(QueryDetailsMetrics m) {
+        this.minTime = m.minimumTime();
+        this.maxTime = m.maximumTime();
+        this.avgTime = m.averageTime();
+        this.execs = m.executions();
+        this.fails = m.fails();
+
+        return this;
+    }
+
+    /**
+     * @return Minimum execution time of query.
+     */
+    public long minimumTime() {
+        return minTime;
+    }
+
+    /**
+     * @return Maximum execution time of query.
+     */
+    public long maximumTime() {
+        return maxTime;
+    }
+
+    /**
+     * @return Average execution time of query.
+     */
+    public double averageTime() {
+        return avgTime;
+    }
+
+    /**
+     * @return Number of executions.
+     */
+    public int executions() {
+        return execs;
+    }
+
+    /**
+     * @return Total number of times a query execution failed.
+     */
+    public int fails() {
+        return fails;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(VisorCacheQueryDetailsMetrics.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8033f119/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheQueryMetrics.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheQueryMetrics.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheQueryMetrics.java
index a3d77e5..0635f31 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheQueryMetrics.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheQueryMetrics.java
@@ -48,16 +48,14 @@ public class VisorCacheQueryMetrics implements Serializable, LessNamingBean {
      * @param m Cache query metrics.
      * @return Data transfer object for given cache metrics.
      */
-    public static VisorCacheQueryMetrics from(QueryMetrics m) {
-        VisorCacheQueryMetrics qm = new VisorCacheQueryMetrics();
-
-        qm.minTime = m.minimumTime();
-        qm.maxTime = m.maximumTime();
-        qm.avgTime = m.averageTime();
-        qm.execs = m.executions();
-        qm.fails = m.fails();
-
-        return qm;
+    public VisorCacheQueryMetrics from(QueryMetrics m) {
+        this.minTime = m.minimumTime();
+        this.maxTime = m.maximumTime();
+        this.avgTime = m.averageTime();
+        this.execs = m.executions();
+        this.fails = m.fails();
+
+        return this;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/8033f119/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheQueryMetricsV2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheQueryMetricsV2.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheQueryMetricsV2.java
new file mode 100644
index 0000000..a08a52f
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheQueryMetricsV2.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.visor.cache;
+
+import java.io.Serializable;
+import org.apache.ignite.cache.query.QueryMetrics;
+import org.apache.ignite.internal.LessNamingBean;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * Data transfer object for cache query metrics.
+ */
+public class VisorCacheQueryMetricsV2 extends VisorCacheQueryMetrics {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** {@inheritDoc} */
+    @Override public VisorCacheQueryMetrics from(QueryMetrics m) {
+        VisorCacheQueryMetrics vm = super.from(m);
+
+        // TODO IGNITE-3443 implement.
+
+        return vm;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(VisorCacheQueryMetricsV2.class, this);
+    }
+}