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/09/16 05:50:04 UTC

[50/50] [abbrv] ignite git commit: IGNITE-3443 WIP

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/5f0d68ce
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/5f0d68ce
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/5f0d68ce

Branch: refs/heads/ignite-3443
Commit: 5f0d68ceab2116a977c8ebae9113990af1413fc4
Parents: b2f2793
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Fri Sep 16 12:49:22 2016 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Fri Sep 16 12:49:22 2016 +0700

----------------------------------------------------------------------
 .../ignite/cache/query/QueryDetailsMetrics.java |  34 ++-
 .../apache/ignite/cache/query/QueryMetrics.java |   6 -
 .../processors/cache/GridCacheUtilityKey.java   |   6 +-
 .../cache/query/GridCacheQueryAdapter.java      |   6 +-
 .../query/GridCacheQueryBaseMetricsAdapter.java | 188 ----------------
 .../GridCacheQueryDetailsMetricsAdapter.java    | 136 ++++++++++--
 .../query/GridCacheQueryDetailsMetricsKey.java  |  94 ++++++++
 .../cache/query/GridCacheQueryManager.java      |  26 ++-
 .../query/GridCacheQueryMetricsAdapter.java     | 216 +++++++++----------
 .../visor/cache/VisorCacheQueryMetrics.java     |  20 --
 10 files changed, 361 insertions(+), 371 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5f0d68ce/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
index 0ebe5b7..6f570be 100644
--- 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
@@ -34,6 +34,27 @@ public interface QueryDetailsMetrics {
     public String query();
 
     /**
+     * Gets total number execution of query.
+     *
+     * @return Number of executions.
+     */
+    public int executions();
+
+    /**
+     * Gets number of completed execution of query.
+     *
+     * @return Number of completed executions.
+     */
+    public int completed();
+
+    /**
+     * Gets number of times a query execution failed.
+     *
+     * @return Number of times a query execution failed.
+     */
+    public int fails();
+
+    /**
      * Gets minimum execution time of query.
      *
      * @return Minimum execution time of query.
@@ -55,16 +76,9 @@ public interface QueryDetailsMetrics {
     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.
+     * Gets total time of all query executions.
      *
-     * @return Total number of times a query execution failed.
+     * @return Total time of all query executions.
      */
-    public int fails();
+    public long totalTime();
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/5f0d68ce/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 4bb0caa..d0f0a50 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,7 +17,6 @@
 
 package org.apache.ignite.cache.query;
 
-import java.util.List;
 import org.apache.ignite.internal.processors.cache.query.CacheQuery;
 
 /**
@@ -60,9 +59,4 @@ public interface QueryMetrics {
      * @return Total number of times a query execution failed.
      */
     public int fails();
-
-    /**
-     * 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/5f0d68ce/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtilityKey.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtilityKey.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtilityKey.java
index 48b6b45..8110170 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtilityKey.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtilityKey.java
@@ -27,8 +27,8 @@ public abstract class GridCacheUtilityKey<K extends GridCacheUtilityKey> impleme
     private static final long serialVersionUID = 0L;
 
     /** {@inheritDoc} */
-    @SuppressWarnings("unchecked") @Override
-    public final boolean equals(Object obj) {
+    @SuppressWarnings("unchecked")
+    @Override public final boolean equals(Object obj) {
         return obj == this || obj != null && obj.getClass() == getClass() && equalsx((K)obj);
     }
 
@@ -42,4 +42,4 @@ public abstract class GridCacheUtilityKey<K extends GridCacheUtilityKey> impleme
 
     /** {@inheritDoc} */
     public abstract int hashCode();
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5f0d68ce/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 32d823a..c0fd1c4 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
@@ -152,7 +152,7 @@ public class GridCacheQueryAdapter<T> implements CacheQuery<T> {
 
         log = cctx.logger(getClass());
 
-        metrics = new GridCacheQueryMetricsAdapter(cctx.config().getQueryMetricsHistorySize());
+        metrics = new GridCacheQueryMetricsAdapter();
 
         this.incMeta = false;
         this.clsName = null;
@@ -192,7 +192,7 @@ public class GridCacheQueryAdapter<T> implements CacheQuery<T> {
 
         log = cctx.logger(getClass());
 
-        metrics = new GridCacheQueryMetricsAdapter(cctx.config().getQueryMetricsHistorySize());
+        metrics = new GridCacheQueryMetricsAdapter();
     }
 
     /**
@@ -463,7 +463,7 @@ public class GridCacheQueryAdapter<T> implements CacheQuery<T> {
 
     /** {@inheritDoc} */
     @Override public void resetMetrics() {
-        metrics = new GridCacheQueryMetricsAdapter(cctx.config().getQueryMetricsHistorySize());
+        metrics = new GridCacheQueryMetricsAdapter();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/5f0d68ce/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
deleted file mode 100644
index 672ef19..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryBaseMetricsAdapter.java
+++ /dev/null
@@ -1,188 +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.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 {
-            double 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/5f0d68ce/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
index cbc144e..e658e99 100644
--- 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
@@ -17,29 +17,99 @@
 
 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.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}.
+ * Adapter for {@link QueryDetailsMetrics}.
  */
-public class GridCacheQueryDetailsMetricsAdapter extends GridCacheQueryBaseMetricsAdapter implements QueryDetailsMetrics {
+public class GridCacheQueryDetailsMetricsAdapter implements QueryDetailsMetrics, Externalizable {
     /** */
     private static final long serialVersionUID = 0L;
 
     /** Query type to track metrics. */
     private GridCacheQueryType qryType;
 
-    /** Query text representation. */
+    /** Textual query representation. */
     private String qry;
 
+    /** Number of executions. */
+    private int execs;
+
+    /** Number of completed executions. */
+    private int completed;
+
+    /** Number of fails. */
+    private int fails;
+
+    /** Minimum time of execution. */
+    private long minTime;
+
+    /** Maximum time of execution. */
+    private long maxTime;
+
+    /** Sum of execution time of completed time. */
+    private long totalTime;
+
+    /**
+     * Required by {@link Externalizable}.
+     */
+    public GridCacheQueryDetailsMetricsAdapter() {
+        // No-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param qryType Query type.
+     * @param qry Textual query representation.
+     */
+    public GridCacheQueryDetailsMetricsAdapter(GridCacheQueryType qryType, String qry) {
+        this.qryType = qryType;
+        this.qry = qry;
+    }
+
+    /**
+     * Callback for query execution.
+     *
+     * @param fail {@code True} query executed unsuccessfully {@code false} otherwise.
+     */
+    public void onQueryExecute(boolean fail) {
+        execs += 1;
+
+        if (fail)
+            fails += 1;
+    }
+
+    /**
+     * Callback for completion of query execution.
+     *
+     * @param duration Duration of queue execution.
+     * @param fail {@code True} query executed unsuccessfully {@code false} otherwise.
+     */
+    public void onQueryCompleted(long duration, boolean fail) {
+        if (fail)
+            fails += 1;
+        else {
+            completed += 1;
+
+            totalTime += duration;
+
+            if (minTime == 0 || minTime > duration)
+                minTime = duration;
+
+            if (maxTime < duration)
+                maxTime = duration;
+        }
+    }
+
     /** {@inheritDoc} */
-    @Override  public GridCacheQueryType queryType() {
+    @Override public GridCacheQueryType queryType() {
         return qryType;
     }
 
@@ -48,33 +118,63 @@ public class GridCacheQueryDetailsMetricsAdapter extends GridCacheQueryBaseMetri
         return qry;
     }
 
-    /**
-     * Merge with given metrics.
-     *
-     * @return Copy.
-     */
-    public GridCacheQueryDetailsMetricsAdapter copy() {
-        GridCacheQueryDetailsMetricsAdapter m = new GridCacheQueryDetailsMetricsAdapter();
+    /** {@inheritDoc} */
+    @Override public int executions() {
+        return execs;
+    }
 
-        copy(m);
+    /** {@inheritDoc} */
+    @Override public int completed() {
+        return completed;
+    }
 
-        return m;
+    /** {@inheritDoc} */
+    @Override public int fails() {
+        return execs - completed;
     }
 
     /** {@inheritDoc} */
-    @Override public void writeExternal(ObjectOutput out) throws IOException {
-        super.writeExternal(out);
+    @Override public long minimumTime() {
+        return minTime;
+    }
 
+    /** {@inheritDoc} */
+    @Override public long maximumTime() {
+        return maxTime;
+    }
+
+    /** {@inheritDoc} */
+    @Override public double averageTime() {
+        double val = completed;
+
+        return val > 0 ? totalTime / val : 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long totalTime() {
+        return totalTime;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
         U.writeEnum(out, qryType);
         U.writeString(out, qry);
+        out.writeInt(execs);
+        out.writeInt(completed);
+        out.writeLong(minTime);
+        out.writeLong(maxTime);
+        out.writeLong(totalTime);
     }
 
     /** {@inheritDoc} */
     @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-        super.readExternal(in);
-
         qryType = GridCacheQueryType.fromOrdinal(in.readByte());
         qry = U.readString(in);
+        execs = in.readInt();
+        completed = in.readInt();
+        minTime = in.readLong();
+        maxTime = in.readLong();
+        totalTime = in.readLong();
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/5f0d68ce/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryDetailsMetricsKey.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryDetailsMetricsKey.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryDetailsMetricsKey.java
new file mode 100644
index 0000000..aaa531a
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryDetailsMetricsKey.java
@@ -0,0 +1,94 @@
+/*
+ * 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.internal.processors.cache.GridCacheUtilityKey;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * Key for query details metrics to store in system cache.
+ */
+public class GridCacheQueryDetailsMetricsKey extends GridCacheUtilityKey<GridCacheQueryDetailsMetricsKey> implements Externalizable {
+    /** Query type. */
+    private GridCacheQueryType qryType;
+
+    /** Query text descriptor: SQL, cache name, search text, ... */
+    private String qry;
+
+    /**
+     * Required by {@link Externalizable}.
+     */
+    public GridCacheQueryDetailsMetricsKey() {
+        // No-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param qryType Query type.
+     * @param qry Query text descriptor.
+     */
+    public GridCacheQueryDetailsMetricsKey(GridCacheQueryType qryType, String qry) {
+        this.qryType = qryType;
+        this.qry = qry;
+    }
+
+    /**
+     * @return Query type.
+     */
+    public GridCacheQueryType 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 = GridCacheQueryType.fromOrdinal(in.readByte());
+        qry = U.readString(in);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean equalsx(GridCacheQueryDetailsMetricsKey that) {
+        return qryType == that.qryType && qry.equals(that.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/5f0d68ce/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 10df2f6..ad4bac1 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
@@ -64,6 +64,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheInternal;
 import org.apache.ignite.internal.processors.cache.GridCacheManagerAdapter;
 import org.apache.ignite.internal.processors.cache.GridCacheOffheapSwapEntry;
 import org.apache.ignite.internal.processors.cache.GridCacheSwapEntryImpl;
+import org.apache.ignite.internal.processors.cache.GridCacheUtilityKey;
 import org.apache.ignite.internal.processors.cache.IgniteCacheExpiryPolicy;
 import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
@@ -144,7 +145,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
     private int maxIterCnt;
 
     /** */
-    private volatile GridCacheQueryMetricsAdapter metrics = new GridCacheQueryMetricsAdapter(5); // TODO: IGNITE-3443 take from cfg.
+    private volatile GridCacheQueryMetricsAdapter metrics = new GridCacheQueryMetricsAdapter();
 
     /** */
     private final ConcurrentMap<UUID, RequestFutureMap> qryIters =
@@ -2075,7 +2076,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
      * Resets metrics.
      */
     public void resetMetrics() {
-        metrics = new GridCacheQueryMetricsAdapter(5); // TODO: IGNITE-3443 take from cfg.
+        metrics = new GridCacheQueryMetricsAdapter();
     }
 
     /**
@@ -2092,7 +2093,26 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
      * @param fail {@code true} if execution failed.
      */
     public void onCompleted(GridCacheQueryType qryType, String qry, long duration, boolean fail) {
-        metrics.onQueryCompleted(qryType, qry, duration, fail);
+        metrics.onQueryCompleted(duration, fail);
+
+        // TODO IGNITE-3443 execute in separate thread and use entry processor.
+        try {
+            IgniteInternalCache<GridCacheUtilityKey, GridCacheQueryDetailsMetricsAdapter> cache = cctx.grid().utilityCache();
+
+            GridCacheQueryDetailsMetricsKey key = new GridCacheQueryDetailsMetricsKey(qryType, qry);
+
+            GridCacheQueryDetailsMetricsAdapter val = cache.get(key);
+
+            if (cache == null)
+                val = new GridCacheQueryDetailsMetricsAdapter(qryType, qry);
+
+            // TODO IGNITE-3443 val.onCompleted(duration, fail);
+
+            cache.put(key, val);
+        }
+        catch (IgniteCheckedException e) {
+            throw new IgniteException(e);
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/5f0d68ce/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 8658f2c..370ab93 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,63 +21,110 @@ import java.io.Externalizable;
 import java.io.IOException;
 import java.io.ObjectInput;
 import java.io.ObjectOutput;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.concurrent.ConcurrentMap;
-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.apache.ignite.internal.util.typedef.internal.U;
-import org.jsr166.ConcurrentLinkedHashMap;
+import org.jsr166.LongAdder8;
 
 /**
  * Adapter for {@link QueryMetrics}.
  */
-public class GridCacheQueryMetricsAdapter extends GridCacheQueryBaseMetricsAdapter implements QueryMetrics {
+public class GridCacheQueryMetricsAdapter implements QueryMetrics, Externalizable {
     /** */
     private static final long serialVersionUID = 0L;
 
-    /** History size. */
-    private final int detailsHistSz;
+    /** Minimum time of execution. */
+    private final GridAtomicLong minTime = new GridAtomicLong();
 
-    /** Map with metrics history for latest queries. */
-    private final ConcurrentMap<QueryMetricsKey, GridCacheQueryDetailsMetricsAdapter> details;
+    /** Maximum time of execution. */
+    private final GridAtomicLong maxTime = new GridAtomicLong();
 
-    /**
-     * @param detailsHistSz Query metrics history size.
+    /** 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.
      */
-    public GridCacheQueryMetricsAdapter(int detailsHistSz) {
-        this.detailsHistSz = detailsHistSz;
+    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();
+
+    /** {@inheritDoc} */
+    @Override public long minimumTime() {
+        return minTime.get();
+    }
 
-        details = new ConcurrentLinkedHashMap<>(detailsHistSz, 0.75f, 16, detailsHistSz > 0 ? detailsHistSz : 1);
+    /** {@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 List<QueryDetailsMetrics> details() {
-        return detailsHistSz > 0 ? new ArrayList<QueryDetailsMetrics>(details.values()) : Collections.<QueryDetailsMetrics>emptyList();
+    @Override public int executions() {
+        return execs.intValue();
     }
 
     /**
-     * Callback for completion of query execution.
+     * 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();
+    }
+
+    /** {@inheritDoc} */
+    @Override public int fails() {
+        return fails.intValue();
+    }
+
+    /**
+     * Callback for 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(GridCacheQueryType qryType, String qry, long duration, boolean fail) {
-        onQueryCompleted(duration, fail);
+    public void onQueryExecute(boolean fail) {
+        execs.increment();
 
-        QueryMetricsKey key = new QueryMetricsKey(qryType, qry);
+        if (fail)
+            fails.increment();
+    }
 
-        if (detailsHistSz > 0) {
-            if (!details.containsKey(key))
-                details.putIfAbsent(key, new GridCacheQueryDetailsMetricsAdapter());
+    /**
+     * Callback for completion of query execution.
+     *
+     * @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);
 
-            GridCacheQueryDetailsMetricsAdapter dm = details.get(key);
+        if (fail)
+            fails.increment();
+        else {
+            completed.increment();
 
-            dm.onQueryCompleted(duration, fail);
+            sumTime.add(duration);
         }
     }
 
@@ -87,111 +134,40 @@ public class GridCacheQueryMetricsAdapter extends GridCacheQueryBaseMetricsAdapt
      * @return Copy.
      */
     public GridCacheQueryMetricsAdapter copy() {
-        GridCacheQueryMetricsAdapter m = new GridCacheQueryMetricsAdapter(detailsHistSz);
+        GridCacheQueryMetricsAdapter m = new GridCacheQueryMetricsAdapter();
 
         // Not synchronized because accuracy isn't critical.
-        copy(m);
-
-        if (detailsHistSz > 0)
-            m.details.putAll(details);
+        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;
 
         return m;
     }
 
     /** {@inheritDoc} */
     @Override public void writeExternal(ObjectOutput out) throws IOException {
-        super.writeExternal(out);
-
-        U.writeMap(out, details);
+        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 {
-        super.readExternal(in);
-
-        details.putAll(U.<QueryMetricsKey, GridCacheQueryDetailsMetricsAdapter>readMap(in));
+        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(GridCacheQueryMetricsAdapter.class, this);
     }
-
-    /**
-     * Key for query metrics to store in map.
-     */
-    private static class QueryMetricsKey implements Externalizable {
-        /** Query type. */
-        private GridCacheQueryType 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(GridCacheQueryType qryType, String qry) {
-            this.qryType = qryType;
-            this.qry = qry;
-        }
-
-        /**
-         * @return Query type.
-         */
-        public GridCacheQueryType 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 = GridCacheQueryType.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/5f0d68ce/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 a7e1deb..15ab1ae 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
@@ -17,9 +17,6 @@
 
 package org.apache.ignite.internal.visor.cache;
 
-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.typedef.internal.S;
 
@@ -30,9 +27,6 @@ public class VisorCacheQueryMetrics extends VisorCacheQueryBaseMetrics {
     /** */
     private static final long serialVersionUID = 0L;
 
-    /** List of query metrics aggregated by query type and textual representation. */
-    private List<VisorCacheQueryDetailsMetrics> details;
-
     /**
      * @param m Cache query metrics.
      * @return Data transfer object for given cache metrics.
@@ -40,23 +34,9 @@ public class VisorCacheQueryMetrics extends VisorCacheQueryBaseMetrics {
     public VisorCacheQueryMetrics from(QueryMetrics m) {
         init(m.minimumTime(), m.maximumTime(), m.averageTime(), m.executions(), m.fails());
 
-        List<QueryDetailsMetrics> mds = m.details();
-
-        details = new ArrayList<>(mds.size());
-
-        for (QueryDetailsMetrics md : mds)
-            details.add(new VisorCacheQueryDetailsMetrics().from(md));
-
         return this;
     }
 
-    /**
-     * @return List of query metrics aggregated by query type and textual representation.
-     */
-    public List<VisorCacheQueryDetailsMetrics> details() {
-        return details;
-    }
-
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(VisorCacheQueryMetrics.class, this);