You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2014/12/19 11:04:16 UTC

[08/11] incubator-ignite git commit: # GG-9572 Refactoring VisorCacheMetrics2

# GG-9572 Refactoring VisorCacheMetrics2


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

Branch: refs/heads/ignite-41
Commit: 3032bee854dcf782362360c15c41c86404cb6a4a
Parents: ce625dc
Author: AKuznetsov <ak...@gridgain.com>
Authored: Fri Dec 19 10:30:00 2014 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Fri Dec 19 10:30:00 2014 +0700

----------------------------------------------------------------------
 .../grid/kernal/visor/cache/VisorCache.java     |   4 +-
 .../cache/VisorCacheAggregatedMetrics.java      | 417 ++++++++++++-------
 .../kernal/visor/cache/VisorCacheMetrics.java   | 176 ++------
 .../kernal/visor/cache/VisorCacheMetrics2.java  | 218 ----------
 .../cache/VisorCacheMetricsCollectorTask.java   | 104 +----
 .../cache/VisorCacheQueryAggregatedMetrics.java | 129 ------
 .../visor/cache/VisorCacheQueryMetrics.java     |  44 +-
 .../visor/node/VisorGridConfiguration.java      |   2 +-
 .../VisorNodeConfigurationCollectorJob.java     |   2 +-
 .../commands/cache/VisorCacheCommand.scala      | 127 +++---
 10 files changed, 404 insertions(+), 819 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3032bee8/modules/core/src/main/java/org/gridgain/grid/kernal/visor/cache/VisorCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/visor/cache/VisorCache.java b/modules/core/src/main/java/org/gridgain/grid/kernal/visor/cache/VisorCache.java
index 9dcf4a1..3ad75dc 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/visor/cache/VisorCache.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/visor/cache/VisorCache.java
@@ -215,9 +215,7 @@ public class VisorCache implements Serializable {
         cache.partitions(ca.affinity().partitions());
         cache.primaryPartitions(pps);
         cache.backupPartitions(bps);
-        cache.metrics(VisorCacheMetrics.from(ca.metrics()));
-//        cache.drSendMetrics(VisorDrSenderCacheMetrics.from(ca));
-//        cache.drReceiveMetrics(VisorDrReceiverCacheMetrics.from(ca));
+        cache.metrics(VisorCacheMetrics.from(ca));
         cache.partitionMap(partsMap);
 
         return cache;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3032bee8/modules/core/src/main/java/org/gridgain/grid/kernal/visor/cache/VisorCacheAggregatedMetrics.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/visor/cache/VisorCacheAggregatedMetrics.java b/modules/core/src/main/java/org/gridgain/grid/kernal/visor/cache/VisorCacheAggregatedMetrics.java
index 2970487..d7a6ff0 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/visor/cache/VisorCacheAggregatedMetrics.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/visor/cache/VisorCacheAggregatedMetrics.java
@@ -24,65 +24,77 @@ public class VisorCacheAggregatedMetrics implements Serializable {
     /** Cache name. */
     private final String cacheName;
 
-    /** Nodes with this cache. */
-    private Collection<UUID> nodes = new ArrayList<>();
+    /** Node IDs with cache metrics. */
+    private Map<UUID, VisorCacheMetrics> metrics = new HashMap<>();
 
-    /** Min number of elements in the cache. */
-    private int minSize = Integer.MAX_VALUE;
+    /** Minimum number of elements in the cache. */
+    private transient Integer minSize;
 
-    /** Avg number of elements in the cache. */
-    private double avgSize;
+    /** Average number of elements in the cache. */
+    private transient Double avgSize;
 
-    /** Max number of elements in the cache. */
-    private int maxSize = Integer.MIN_VALUE;
+    /** Maximum number of elements in the cache. */
+    private transient Integer maxSize;
 
     /** Gets last read time of the owning cache. */
-    private long lastRead;
+    private transient Long lastRead;
 
     /** Gets last read time of the owning cache. */
-    private long lastWrite;
+    private transient Long lastWrite;
 
-    /** Hits of the owning cache. */
-    private int minHits = Integer.MAX_VALUE;
+    /** Minimum hits of the owning cache. */
+    private transient Integer minHits;
 
-    /** Hits of the owning cache. */
-    private double avgHits;
+    /** Average hits of the owning cache. */
+    private transient Double avgHits;
 
-    /** Hits of the owning cache. */
-    private int maxHits = Integer.MIN_VALUE;
+    /** Maximum hits of the owning cache. */
+    private transient Integer maxHits;
 
     /** Minimum misses of the owning cache. */
-    private int minMisses = Integer.MAX_VALUE;
+    private transient Integer minMisses;
 
     /** Average misses of the owning cache. */
-    private double avgMisses;
+    private transient Double avgMisses;
 
     /** Maximum misses of the owning cache. */
-    private int maxMisses = Integer.MIN_VALUE;
+    private transient Integer maxMisses;
 
     /** Minimum total number of reads of the owning cache. */
-    private int minReads = Integer.MAX_VALUE;
+    private transient Integer minReads;
 
     /** Average total number of reads of the owning cache. */
-    private double avgReads;
+    private transient Double avgReads;
 
     /** Maximum total number of reads of the owning cache. */
-    private int maxReads = Integer.MIN_VALUE;
+    private transient Integer maxReads;
 
     /** Minimum total number of writes of the owning cache. */
-    private int minWrites = Integer.MAX_VALUE;
+    private transient Integer minWrites;
 
     /** Average total number of writes of the owning cache. */
-    private double avgWrites;
+    private transient Double avgWrites;
 
     /** Maximum total number of writes of the owning cache. */
-    private int maxWrites = Integer.MIN_VALUE;
+    private transient Integer maxWrites;
 
-    /**  */
-    private VisorCacheQueryAggregatedMetrics qryMetrics = new VisorCacheQueryAggregatedMetrics();
+    /** Minimum execution time of query. */
+    private transient Long minQryTime;
 
-    /**  */
-    private Collection<VisorCacheMetrics2> metrics = new ArrayList<>();
+    /** Average execution time of query. */
+    private transient Double avgQryTime;
+
+    /** Maximum execution time of query. */
+    private transient Long maxQryTime;
+
+    /** Total execution time of query. */
+    private transient Long totalQryTime;
+
+    /** Number of executions. */
+    private transient Integer execsQry;
+
+    /** Total number of times a query execution failed. */
+    private transient Integer failsQry;
 
     /**
      * Create data transfer object with given parameters.
@@ -104,258 +116,343 @@ public class VisorCacheAggregatedMetrics implements Serializable {
      * @return Nodes.
      */
     public Collection<UUID> nodes() {
-        return nodes;
+        return metrics.keySet();
     }
 
     /**
-     * @return Min size.
+     * @return Minimum number of elements in the cache.
      */
-    public int minSize() {
+    public int minimumSize() {
+        if (minSize == null) {
+            minSize = Integer.MAX_VALUE;
+
+            for (VisorCacheMetrics metric : metrics.values())
+                minSize = Math.min(minSize, metric.size());
+        }
+
         return minSize;
     }
 
     /**
-     * @param size New min size.
+     * @return Average number of elements in the cache.
      */
-    public void minSize(int size) {
-        minSize = Math.min(minSize, size);
-    }
+    public double averageSize() {
+        if (avgSize == null) {
+            avgSize = 0.0d;
+
+            for (VisorCacheMetrics metric : metrics.values())
+                avgSize += metric.size();
+
+            avgSize /= metrics.size();
+        }
 
-    /**
-     * @return Avg size.
-     */
-    public double avgSize() {
         return avgSize;
     }
 
     /**
-     * @param avgSize New avg size.
+     * @return Maximum number of elements in the cache.
      */
-    public void avgSize(double avgSize) {
-        this.avgSize = avgSize;
-    }
+    public int maximumSize() {
+        if (maxSize == null) {
+            maxSize = Integer.MIN_VALUE;
 
-    /**
-     * @return Max size.
-     */
-    public int maxSize() {
-        return maxSize;
-    }
+            for (VisorCacheMetrics metric : metrics.values())
+                maxSize = Math.max(maxSize, metric.size());
+        }
 
-    /**
-     * @param size New max size.
-     */
-    public void maxSize(int size) {
-        maxSize = Math.max(maxSize, size);
+        return maxSize;
     }
 
     /**
-     * @return Last read.
+     * @return Last read time of the owning cache.
      */
     public long lastRead() {
-        return lastRead;
-    }
+        if (lastRead == null) {
+            lastRead = Long.MIN_VALUE;
 
-    /**
-     * @param lastRead New last read.
-     */
-    public void lastRead(long lastRead) {
-        this.lastRead = Math.max(this.lastRead, lastRead);
+            for (VisorCacheMetrics metric : metrics.values())
+                lastRead = Math.max(lastRead, metric.readTime());
+        }
+
+        return lastRead;
     }
 
     /**
-     * @return Last write.
+     * @return Last read time of the owning cache.
      */
     public long lastWrite() {
+        if (lastWrite == null) {
+            lastWrite = Long.MIN_VALUE;
+
+            for (VisorCacheMetrics metric : metrics.values())
+                lastWrite = Math.max(lastWrite, metric.readTime());
+        }
+
         return lastWrite;
     }
 
     /**
-     * @param lastWrite New last write.
+     * @return Minimum hits of the owning cache.
      */
-    public void lastWrite(long lastWrite) {
-        this.lastWrite = Math.max(this.lastWrite, lastWrite);
-    }
+    public int minimumHits() {
+        if (minHits == null) {
+            minHits = Integer.MAX_VALUE;
+
+            for (VisorCacheMetrics metric : metrics.values())
+                minHits = Math.min(minHits, metric.hits());
+        }
 
-    /**
-     * @return Min hits.
-     */
-    public int minHits() {
         return minHits;
     }
 
     /**
-     * @param minHits New min hits.
+     * @return Average hits of the owning cache.
      */
-    public void minHits(int minHits) {
-        this.minHits = Math.min(this.minHits, minHits);
-    }
+    public double averageHits() {
+        if (avgHits == null) {
+            avgHits = 0.0d;
+
+            for (VisorCacheMetrics metric : metrics.values())
+                avgHits += metric.hits();
+
+            avgHits /= metrics.size();
+        }
 
-    /**
-     * @return Avg hits.
-     */
-    public double avgHits() {
         return avgHits;
     }
 
     /**
-     * @param avgHits New avg hits.
+     * @return Maximum hits of the owning cache.
      */
-    public void avgHits(double avgHits) {
-        this.avgHits = avgHits;
-    }
+    public int maximumHits() {
+        if (maxHits == null) {
+            maxHits = Integer.MIN_VALUE;
+
+            for (VisorCacheMetrics metric : metrics.values())
+                maxHits = Math.max(maxHits, metric.hits());
+        }
 
-    /**
-     * @return Max hits.
-     */
-    public int maxHits() {
         return maxHits;
     }
 
     /**
-     * @param hits New max hits.
+     * @return Minimum misses of the owning cache.
      */
-    public void maxHits(int hits) {
-        maxHits = Math.max(maxHits, hits);
-    }
+    public int minimumMisses() {
+        if (minMisses == null) {
+            minMisses = Integer.MAX_VALUE;
+
+            for (VisorCacheMetrics metric : metrics.values())
+                minMisses = Math.min(minMisses, metric.misses());
+        }
 
-    /**
-     * @return Min misses.
-     */
-    public int minMisses() {
         return minMisses;
     }
 
     /**
-     * @param misses New min misses.
+     * @return Average misses of the owning cache.
      */
-    public void minMisses(int misses) {
-        minMisses = Math.min(minMisses, misses);
-    }
+    public double averageMisses() {
+        if (avgMisses == null) {
+            avgMisses = 0.0d;
+
+            for (VisorCacheMetrics metric : metrics.values())
+                avgMisses += metric.misses();
+
+            avgMisses /= metrics.size();
+        }
 
-    /**
-     * @return Avg misses.
-     */
-    public double avgMisses() {
         return avgMisses;
     }
 
     /**
-     * @param avgMisses New avg misses.
+     * @return Maximum misses of the owning cache.
      */
-    public void avgMisses(double avgMisses) {
-        this.avgMisses = avgMisses;
-    }
+    public int maximumMisses() {
+        if (maxMisses == null) {
+            maxMisses = Integer.MIN_VALUE;
+
+            for (VisorCacheMetrics metric : metrics.values())
+                maxMisses = Math.max(maxMisses, metric.misses());
+        }
 
-    /**
-     * @return Max misses.
-     */
-    public int maxMisses() {
         return maxMisses;
     }
 
     /**
-     * @param maxMisses New max misses.
+     * @return Minimum total number of reads of the owning cache.
      */
-    public void maxMisses(int maxMisses) {
-        this.maxMisses = maxMisses;
-    }
+    public int minimumReads() {
+        if (minReads == null) {
+            minReads = Integer.MAX_VALUE;
+
+            for (VisorCacheMetrics metric : metrics.values())
+                minReads = Math.min(minReads, metric.reads());
+        }
 
-    /**
-     * @return Min reads.
-     */
-    public int minReads() {
         return minReads;
     }
 
     /**
-     * @param reads New min reads.
+     * @return Average total number of reads of the owning cache.
      */
-    public void minReads(int reads) {
-        minReads = Math.min(minReads, reads);
-    }
+    public double averageReads() {
+        if (avgReads == null) {
+            avgReads = 0.0d;
+
+            for (VisorCacheMetrics metric : metrics.values())
+                avgReads += metric.reads();
+
+            avgReads /= metrics.size();
+        }
 
-    /**
-     * @return Avg reads.
-     */
-    public double avgReads() {
         return avgReads;
     }
 
     /**
-     * @param avgReads New avg reads.
+     * @return Maximum total number of reads of the owning cache.
      */
-    public void avgReads(double avgReads) {
-        this.avgReads = avgReads;
+    public int maximumReads() {
+        if (maxReads == null) {
+            maxReads = Integer.MIN_VALUE;
+
+            for (VisorCacheMetrics metric : metrics.values())
+                maxReads = Math.max(maxReads, metric.reads());
+        }
+
+        return maxReads;
     }
 
     /**
-     * @return Max reads.
+     * @return Minimum total number of writes of the owning cache.
      */
-    public int maxReads() {
-        return maxReads;
+    public int minimumWrites() {
+        if (minWrites == null) {
+            minWrites = Integer.MAX_VALUE;
+
+            for (VisorCacheMetrics metric : metrics.values())
+                minWrites = Math.min(minWrites, metric.writes());
+        }
+
+        return minWrites;
     }
 
     /**
-     * @param reads New max reads.
+     * @return Average total number of writes of the owning cache.
      */
-    public void maxReads(int reads) {
-        maxReads = Math.max(maxReads, reads);
+    public double averageWrites() {
+        if (avgWrites == null) {
+            avgWrites = 0.0d;
+
+            for (VisorCacheMetrics metric : metrics.values())
+                avgWrites += metric.writes();
+
+            avgWrites /= metrics.size();
+        }
+
+        return avgWrites;
     }
 
     /**
-     * @return Min writes.
+     * @return Maximum total number of writes of the owning cache.
      */
-    public int minWrites() {
-        return minWrites;
+    public int maximumWrites() {
+        if (maxWrites == null) {
+            maxWrites = Integer.MIN_VALUE;
+
+            for (VisorCacheMetrics metric : metrics.values())
+                maxWrites = Math.max(maxWrites, metric.writes());
+        }
+
+        return maxWrites;
     }
 
     /**
-     * @param writes New min writes.
+     * @return Minimum execution time of query.
      */
-    public void minWrites(int writes) {
-        minWrites = Math.min(minWrites, writes);
+    public long minimumQueryTime() {
+        if (minQryTime == null) {
+            minQryTime = Long.MAX_VALUE;
+
+            for (VisorCacheMetrics metric : metrics.values())
+                minQryTime = Math.min(minQryTime, metric.queryMetrics().minimumTime());
+        }
+
+        return minQryTime;
     }
 
     /**
-     * @return Avg writes.
+     * @return Average execution time of query.
      */
-    public double avgWrites() {
-        return avgWrites;
+    public double averageQueryTime() {
+        if (avgQryTime == null) {
+            avgQryTime = 0.0d;
+
+            for (VisorCacheMetrics metric : metrics.values())
+                avgQryTime += metric.queryMetrics().averageTime();
+
+            avgQryTime /= metrics.size();
+        }
+
+        return avgQryTime;
     }
 
     /**
-     * @param avgWrites New avg writes.
+     * @return Maximum execution time of query.
      */
-    public void avgWrites(double avgWrites) {
-        this.avgWrites = avgWrites;
+    public long maximumQueryTime() {
+        if (maxQryTime == null) {
+            maxQryTime = Long.MIN_VALUE;
+
+            for (VisorCacheMetrics metric : metrics.values())
+                maxQryTime = Math.max(maxQryTime, metric.queryMetrics().maximumTime());
+        }
+
+        return maxQryTime;
     }
 
     /**
-     * @return Max writes.
+     * @return Total execution time of query.
      */
-    public int maxWrites() {
-        return maxWrites;
+    public long totalQueryTime() {
+        if (totalQryTime == null)
+            totalQryTime = (long)(averageQueryTime() * execsQuery());
+
+        return totalQryTime;
     }
 
     /**
-     * @param writes New max writes.
+     * @return Number of executions.
      */
-    public void maxWrites(int writes) {
-        maxWrites = Math.max(maxWrites, writes);
+    public int execsQuery() {
+        if (execsQry == null) {
+            execsQry = 0;
+
+            for (VisorCacheMetrics metric : metrics.values())
+                execsQry = execsQry + metric.queryMetrics().executions();
+        }
+
+        return execsQry;
     }
 
     /**
-     * @return Query metrics.
+     * @return Total number of times a query execution failed.
      */
-    public VisorCacheQueryAggregatedMetrics queryMetrics() {
-        return qryMetrics;
+    public int failsQuery() {
+        if (failsQry == null) {
+            failsQry = 0;
+
+            for (VisorCacheMetrics metric : metrics.values())
+                failsQry = failsQry + metric.queryMetrics().fails();
+        }
+
+        return failsQry;
     }
 
     /**
-     * @return Metrics.
+     * @return Node IDs with cache metrics.
      */
-    public Collection<VisorCacheMetrics2> metrics() {
+    public Map<UUID, VisorCacheMetrics> metrics() {
         return metrics;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3032bee8/modules/core/src/main/java/org/gridgain/grid/kernal/visor/cache/VisorCacheMetrics.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/visor/cache/VisorCacheMetrics.java b/modules/core/src/main/java/org/gridgain/grid/kernal/visor/cache/VisorCacheMetrics.java
index b49c557..5b75c59 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/visor/cache/VisorCacheMetrics.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/visor/cache/VisorCacheMetrics.java
@@ -21,6 +21,9 @@ public class VisorCacheMetrics implements Serializable {
     /** */
     private static final long serialVersionUID = 0L;
 
+    /** Gets the number of all entries cached on this node. */
+    private int size;
+
     /** Create time of the owning entity (either cache or entry). */
     private long createTm;
 
@@ -72,6 +75,9 @@ public class VisorCacheMetrics implements Serializable {
     /** Rollbacks per second. */
     private int rollbacksPerSec;
 
+    /** Gets query metrics for cache. */
+    private VisorCacheQueryMetrics qryMetrics;
+
     /** Calculate rate of metric per second. */
     private static int perSecond(int metric, long time, long createTime) {
         long seconds = (time - createTime) / 1000;
@@ -80,38 +86,41 @@ public class VisorCacheMetrics implements Serializable {
     }
 
     /**
-     * @param m Cache metrics.
+     * @param c Cache.
      * @return Data transfer object for given cache metrics.
      */
-    public static VisorCacheMetrics from(GridCacheMetrics m) {
+    public static VisorCacheMetrics from(GridCache c) {
+        VisorCacheMetrics cm = new VisorCacheMetrics();
+
         // TODO gg-9141
+        GridCacheMetrics m = c.metrics();
 
-        assert m != null;
+        cm.size = c.size();
 
-        VisorCacheMetrics metrics = new VisorCacheMetrics();
+        cm.createTm = m.createTime();
+        cm.writeTm = m.writeTime();
+        cm.readTm = m.readTime();
+//        cm.commitTm = m.commitTime();
+//        cm.rollbackTm = m.rollbackTime();
 
-        metrics.createTime(m.createTime());
-        metrics.writeTime(m.writeTime());
-        metrics.readTime(m.readTime());
-//        metrics.commitTime(m.commitTime());
-//        metrics.rollbackTime(m.rollbackTime());
+        cm.reads = m.reads();
+        cm.writes = m.writes();
+        cm.hits = m.hits();
+        cm.misses = m.misses();
 
-        metrics.reads(m.reads());
-        metrics.writes(m.writes());
-        metrics.hits(m.hits());
-        metrics.misses(m.misses());
+//        cm.txCommits = m.txCommits();
+//        cm.txRollbacks = m.txRollbacks();
 
-//        metrics.txCommits(m.txCommits());
-//        metrics.txRollbacks(m.txRollbacks());
+        cm.readsPerSec = perSecond(m.reads(), m.readTime(), m.createTime());
+        cm.writesPerSec = perSecond(m.writes(), m.writeTime(), m.createTime());
+        cm.hitsPerSec = perSecond (m.hits(), m.readTime(), m.createTime());
+        cm.missesPerSec = perSecond(m.misses(), m.readTime(), m.createTime());
+//        cm.commitsPerSec = perSecond(m.txCommits(), m.commitTime(), m.createTime());
+//        cm.rollbacksPerSec = perSecond(m.txRollbacks(), m.rollbackTime(), m.createTime());
 
-        metrics.readsPerSecond(perSecond(m.reads(), m.readTime(), m.createTime()));
-        metrics.writesPerSecond(perSecond(m.writes(), m.writeTime(), m.createTime()));
-        metrics.hitsPerSecond(perSecond(m.hits(), m.readTime(), m.createTime()));
-        metrics.missesPerSecond(perSecond(m.misses(), m.readTime(), m.createTime()));
-//        metrics.commitsPerSecond(perSecond(m.txCommits(), m.commitTime(), m.createTime()));
-//        metrics.rollbacksPerSecond(perSecond(m.txRollbacks(), m.rollbackTime(), m.createTime()));
+        cm.qryMetrics = VisorCacheQueryMetrics.from(c.queries().metrics());
 
-        return metrics;
+        return cm;
     }
 
     /**
@@ -122,13 +131,6 @@ public class VisorCacheMetrics implements Serializable {
     }
 
     /**
-     * @param createTm New create time of the owning entity (either cache or entry).
-     */
-    public void createTime(long createTm) {
-        this.createTm = createTm;
-    }
-
-    /**
      * @return Last write time of the owning entity (either cache or entry).
      */
     public long writeTime() {
@@ -136,13 +138,6 @@ public class VisorCacheMetrics implements Serializable {
     }
 
     /**
-     * @param writeTm New last write time of the owning entity (either cache or entry).
-     */
-    public void writeTime(long writeTm) {
-        this.writeTm = writeTm;
-    }
-
-    /**
      * @return Last read time of the owning entity (either cache or entry).
      */
     public long readTime() {
@@ -150,13 +145,6 @@ public class VisorCacheMetrics implements Serializable {
     }
 
     /**
-     * @param readTm New last read time of the owning entity (either cache or entry).
-     */
-    public void readTime(long readTm) {
-        this.readTm = readTm;
-    }
-
-    /**
      * @return Last time transaction was committed.
      */
     public long commitTime() {
@@ -164,13 +152,6 @@ public class VisorCacheMetrics implements Serializable {
     }
 
     /**
-     * @param commitTm New last time transaction was committed.
-     */
-    public void commitTime(long commitTm) {
-        this.commitTm = commitTm;
-    }
-
-    /**
      * @return Last time transaction was rollback.
      */
     public long rollbackTime() {
@@ -178,13 +159,6 @@ public class VisorCacheMetrics implements Serializable {
     }
 
     /**
-     * @param rollbackTm New last time transaction was rollback.
-     */
-    public void rollbackTime(long rollbackTm) {
-        this.rollbackTm = rollbackTm;
-    }
-
-    /**
      * @return Total number of reads of the owning entity (either cache or entry).
      */
     public int reads() {
@@ -192,13 +166,6 @@ public class VisorCacheMetrics implements Serializable {
     }
 
     /**
-     * @param reads New total number of reads of the owning entity (either cache or entry).
-     */
-    public void reads(int reads) {
-        this.reads = reads;
-    }
-
-    /**
      * @return Total number of writes of the owning entity (either cache or entry).
      */
     public int writes() {
@@ -206,13 +173,6 @@ public class VisorCacheMetrics implements Serializable {
     }
 
     /**
-     * @param writes New total number of writes of the owning entity (either cache or entry).
-     */
-    public void writes(int writes) {
-        this.writes = writes;
-    }
-
-    /**
      * @return Total number of hits for the owning entity (either cache or entry).
      */
     public int hits() {
@@ -220,13 +180,6 @@ public class VisorCacheMetrics implements Serializable {
     }
 
     /**
-     * @param hits New total number of hits for the owning entity (either cache or entry).
-     */
-    public void hits(int hits) {
-        this.hits = hits;
-    }
-
-    /**
      * @return Total number of misses for the owning entity (either cache or entry).
      */
     public int misses() {
@@ -234,13 +187,6 @@ public class VisorCacheMetrics implements Serializable {
     }
 
     /**
-     * @param misses New total number of misses for the owning entity (either cache or entry).
-     */
-    public void misses(int misses) {
-        this.misses = misses;
-    }
-
-    /**
      * @return Total number of transaction commits.
      */
     public int txCommits() {
@@ -248,13 +194,6 @@ public class VisorCacheMetrics implements Serializable {
     }
 
     /**
-     * @param txCommits New total number of transaction commits.
-     */
-    public void txCommits(int txCommits) {
-        this.txCommits = txCommits;
-    }
-
-    /**
      * @return Total number of transaction rollbacks.
      */
     public int txRollbacks() {
@@ -262,13 +201,6 @@ public class VisorCacheMetrics implements Serializable {
     }
 
     /**
-     * @param txRollbacks New total number of transaction rollbacks.
-     */
-    public void txRollbacks(int txRollbacks) {
-        this.txRollbacks = txRollbacks;
-    }
-
-    /**
      * @return Reads per second.
      */
     public int readsPerSecond() {
@@ -276,13 +208,6 @@ public class VisorCacheMetrics implements Serializable {
     }
 
     /**
-     * @param readsPerSec New reads per second.
-     */
-    public void readsPerSecond(int readsPerSec) {
-        this.readsPerSec = readsPerSec;
-    }
-
-    /**
      * @return Writes per second.
      */
     public int writesPerSecond() {
@@ -290,13 +215,6 @@ public class VisorCacheMetrics implements Serializable {
     }
 
     /**
-     * @param writesPerSec New writes per second.
-     */
-    public void writesPerSecond(int writesPerSec) {
-        this.writesPerSec = writesPerSec;
-    }
-
-    /**
      * @return Hits per second.
      */
     public int hitsPerSecond() {
@@ -304,13 +222,6 @@ public class VisorCacheMetrics implements Serializable {
     }
 
     /**
-     * @param hitsPerSec New hits per second.
-     */
-    public void hitsPerSecond(int hitsPerSec) {
-        this.hitsPerSec = hitsPerSec;
-    }
-
-    /**
      * @return Misses per second.
      */
     public int missesPerSecond() {
@@ -318,13 +229,6 @@ public class VisorCacheMetrics implements Serializable {
     }
 
     /**
-     * @param missesPerSec New misses per second.
-     */
-    public void missesPerSecond(int missesPerSec) {
-        this.missesPerSec = missesPerSec;
-    }
-
-    /**
      * @return Commits per second.
      */
     public int commitsPerSecond() {
@@ -332,24 +236,24 @@ public class VisorCacheMetrics implements Serializable {
     }
 
     /**
-     * @param commitsPerSec New commits per second.
+     * @return Rollbacks per second.
      */
-    public void commitsPerSecond(int commitsPerSec) {
-        this.commitsPerSec = commitsPerSec;
+    public int rollbacksPerSecond() {
+        return rollbacksPerSec;
     }
 
     /**
-     * @return Rollbacks per second.
+     * @return Gets the number of all entries cached on this node.
      */
-    public int rollbacksPerSecond() {
-        return rollbacksPerSec;
+    public int size() {
+        return size;
     }
 
     /**
-     * @param rollbacksPerSec New rollbacks per second.
+     * @return Gets query metrics for cache.
      */
-    public void rollbacksPerSecond(int rollbacksPerSec) {
-        this.rollbacksPerSec = rollbacksPerSec;
+    public VisorCacheQueryMetrics queryMetrics() {
+        return qryMetrics;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3032bee8/modules/core/src/main/java/org/gridgain/grid/kernal/visor/cache/VisorCacheMetrics2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/visor/cache/VisorCacheMetrics2.java b/modules/core/src/main/java/org/gridgain/grid/kernal/visor/cache/VisorCacheMetrics2.java
deleted file mode 100644
index 626d127..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/visor/cache/VisorCacheMetrics2.java
+++ /dev/null
@@ -1,218 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.kernal.visor.cache;
-
-import org.gridgain.grid.util.typedef.internal.*;
-
-import java.io.*;
-import java.util.*;
-
-/**
- * Data transfer object for cache metrics.
- */
-public class VisorCacheMetrics2 implements Serializable {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** Cache name. */
-    private final String cacheName;
-
-    /** Node id. */
-    private final UUID nodeId;
-
-    /** Number of CPUs available to the JVM. */
-    private final int cpus;
-
-    /** Percent of heap memory used. */
-    private final double heapUsed;
-
-    /** Percent of estimated CPU usage. */
-    private final double cpuLoad;
-
-    /** Uptime of the JVM in milliseconds. */
-    private final long upTime;
-
-    /** Number of elements in the cache. */
-    private final int size;
-
-    /** Last read time of the owning cache. */
-    private final long lastRead;
-
-    /** Last write time of the owning cache. */
-    private final long lastWrite;
-
-    /** Hits of the owning cache. */
-    private final int hits;
-
-    /** Misses of the owning cache. */
-    private final int misses;
-
-    /** Total number of reads of the owning cache. */
-    private final int reads;
-
-    /** Total number of writes of the owning cache. */
-    private final int writes;
-
-    /** Gets query metrics for cache. */
-    private final VisorCacheQueryMetrics qryMetrics;
-
-    /**
-     * Create data transfer object with given parameters.
-     *
-     * @param cacheName Cache name.
-     * @param nodeId Node id.
-     * @param cpus Number of CPUs available to the JVM
-     * @param heapUsed Percent of heap memory used.
-     * @param cpuLoad Percent of estimated CPU usage.
-     * @param upTime Uptime of the JVM in milliseconds.
-     * @param size Number of elements in the cache.
-     * @param lastRead Last read time of the owning cache.
-     * @param lastWrite Last write time of the owning cache.
-     * @param hits Hits of the owning cache.
-     * @param misses Misses of the owning cache.
-     * @param reads Total number of reads of the owning cache.
-     * @param writes Total number of writes of the owning cache.
-     * @param qryMetrics Gets query metrics for cache.
-     */
-    public VisorCacheMetrics2(
-        String cacheName,
-        UUID nodeId,
-        int cpus,
-        double heapUsed,
-        double cpuLoad,
-        long upTime,
-        int size,
-        long lastRead,
-        long lastWrite,
-        int hits,
-        int misses,
-        int reads,
-        int writes,
-        VisorCacheQueryMetrics qryMetrics
-    ) {
-        this.cacheName = cacheName;
-        this.nodeId = nodeId;
-        this.cpus = cpus;
-        this.heapUsed = heapUsed;
-        this.cpuLoad = cpuLoad;
-        this.upTime = upTime;
-        this.size = size;
-        this.lastRead = lastRead;
-        this.lastWrite = lastWrite;
-        this.hits = hits;
-        this.misses = misses;
-        this.reads = reads;
-        this.writes = writes;
-        this.qryMetrics = qryMetrics;
-    }
-
-    /**
-     * @return Cache name.
-     */
-    public String cacheName() {
-        return cacheName;
-    }
-
-    /**
-     * @return Node id.
-     */
-    public UUID nodeId() {
-        return nodeId;
-    }
-
-    /**
-     * @return Number of CPUs available to the JVM.
-     */
-    public int cpus() {
-        return cpus;
-    }
-
-    /**
-     * @return Percent of heap memory used.
-     */
-    public double heapUsed() {
-        return heapUsed;
-    }
-
-    /**
-     * @return Percent of estimated CPU usage.
-     */
-    public double cpuLoad() {
-        return cpuLoad;
-    }
-
-    /**
-     * @return Uptime of the JVM in milliseconds.
-     */
-    public long upTime() {
-        return upTime;
-    }
-
-    /**
-     * @return Number of elements in the cache.
-     */
-    public int size() {
-        return size;
-    }
-
-    /**
-     * @return Gets last read time of the owning cache.
-     */
-    public long lastRead() {
-        return lastRead;
-    }
-
-    /**
-     * @return Gets last write time of the owning cache.
-     */
-    public long lastWrite() {
-        return lastWrite;
-    }
-
-    /**
-     * @return Gets hits of the owning cache.
-     */
-    public int hits() {
-        return hits;
-    }
-
-    /**
-     * @return Gets misses of the owning cache.
-     */
-    public int misses() {
-        return misses;
-    }
-
-    /**
-     * @return Gets total number of reads of the owning cache.
-     */
-    public int reads() {
-        return reads;
-    }
-
-    /**
-     * @return Gets total number of writes of the owning cache.
-     */
-    public int writes() {
-        return writes;
-    }
-
-    /**
-     * @return Gets query metrics for cache.
-     */
-    public VisorCacheQueryMetrics queryMetrics() {
-        return qryMetrics;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(VisorCacheMetrics2.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3032bee8/modules/core/src/main/java/org/gridgain/grid/kernal/visor/cache/VisorCacheMetricsCollectorTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/visor/cache/VisorCacheMetricsCollectorTask.java b/modules/core/src/main/java/org/gridgain/grid/kernal/visor/cache/VisorCacheMetricsCollectorTask.java
index cd92579..47390a9 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/visor/cache/VisorCacheMetricsCollectorTask.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/visor/cache/VisorCacheMetricsCollectorTask.java
@@ -10,11 +10,9 @@
 package org.gridgain.grid.kernal.visor.cache;
 
 import org.apache.ignite.*;
-import org.apache.ignite.cluster.*;
 import org.apache.ignite.compute.*;
 import org.apache.ignite.lang.*;
 import org.gridgain.grid.cache.*;
-import org.gridgain.grid.cache.query.*;
 import org.gridgain.grid.kernal.processors.task.*;
 import org.gridgain.grid.kernal.visor.*;
 import org.gridgain.grid.util.typedef.*;
@@ -28,7 +26,7 @@ import java.util.*;
  */
 @GridInternal
 public class VisorCacheMetricsCollectorTask extends VisorMultiNodeTask<IgniteBiTuple<Boolean, String>,
-    Iterable<VisorCacheAggregatedMetrics>, Collection<VisorCacheMetrics2>> {
+    Iterable<VisorCacheAggregatedMetrics>, Map<String, VisorCacheMetrics>> {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -40,80 +38,34 @@ public class VisorCacheMetricsCollectorTask extends VisorMultiNodeTask<IgniteBiT
     /** {@inheritDoc} */
     @Nullable @Override protected Iterable<VisorCacheAggregatedMetrics> reduce0(List<ComputeJobResult> results)
         throws IgniteCheckedException {
-        Map<String, VisorCacheAggregatedMetrics> grpAggrMetrics = new HashMap<>();
+        Map<String, VisorCacheAggregatedMetrics> grpAggrMetrics = U.newHashMap(results.size());
 
         for (ComputeJobResult res : results) {
-            if (res.getException() == null && res.getData() instanceof Collection<?>) {
-                Collection<VisorCacheMetrics2> cms = res.getData();
-                for (VisorCacheMetrics2 cm : cms) {
+            if (res.getException() == null && res.getData() instanceof Map<?, ?>) {
+                Map<String, VisorCacheMetrics> cms = res.getData();
 
-                    VisorCacheAggregatedMetrics am = grpAggrMetrics.get(cm.cacheName());
+                for (Map.Entry<String, VisorCacheMetrics> entry : cms.entrySet()) {
+                    VisorCacheAggregatedMetrics am = grpAggrMetrics.get(entry.getKey());
 
                     if (am == null) {
-                        am = new VisorCacheAggregatedMetrics(cm.cacheName());
+                        am = new VisorCacheAggregatedMetrics(entry.getKey());
 
-                        grpAggrMetrics.put(cm.cacheName(), am);
+                        grpAggrMetrics.put(entry.getKey(), am);
                     }
 
-                    am.nodes().add(cm.nodeId());
-                    am.minSize(cm.size());
-                    am.maxSize(cm.size());
-                    am.lastRead(cm.lastRead());
-                    am.lastWrite(cm.lastWrite());
-                    am.minHits(cm.hits());
-                    am.maxHits(cm.hits());
-                    am.minMisses(cm.misses());
-                    am.maxMisses(cm.misses());
-                    am.minReads(cm.reads());
-                    am.maxReads(cm.reads());
-                    am.minWrites(cm.writes());
-                    am.maxWrites(cm.writes());
-                    am.metrics().add(cm);
-
-                    // Partial aggregation of averages.
-                    am.avgReads(am.avgReads() + cm.reads());
-                    am.avgWrites(am.avgWrites() + cm.writes());
-                    am.avgMisses(am.avgMisses() + cm.misses());
-                    am.avgHits(am.avgHits() + cm.hits());
-                    am.avgSize(am.avgSize() + cm.size());
-
-                    // Aggregate query metrics data
-                    VisorCacheQueryMetrics qm = cm.queryMetrics();
-                    VisorCacheQueryAggregatedMetrics aqm = am.queryMetrics();
-
-                    aqm.minTime(qm.minTime());
-                    aqm.maxTime(qm.maxTime());
-                    aqm.totalTime((long)(aqm.totalTime() + (qm.avgTime() * qm.execs())));
-                    aqm.execs(aqm.execs() + qm.execs());
-                    aqm.fails(aqm.fails() + qm.fails());
+                    am.metrics().put(res.getNode().id(), entry.getValue());
                 }
             }
         }
 
-        Collection<VisorCacheAggregatedMetrics> aggrMetrics = grpAggrMetrics.values();
-
-        // Final aggregation of averages.
-        for (VisorCacheAggregatedMetrics metric : aggrMetrics) {
-            int sz = metric.nodes().size();
-
-            metric.avgSize(metric.avgSize() / sz);
-            metric.avgHits(metric.avgHits() / sz);
-            metric.avgMisses(metric.avgMisses() / sz);
-            metric.avgReads(metric.avgReads() / sz);
-            metric.avgWrites(metric.avgWrites() / sz);
-
-            VisorCacheQueryAggregatedMetrics aqm = metric.queryMetrics();
-
-            aqm.avgTime(aqm.execs() > 0 ? (double)aqm.totalTime() / aqm.execs() : 0.0);
-        }
-
-        return aggrMetrics;
+        return grpAggrMetrics.values();
     }
 
     /**
      * Job that collect cache metrics from node.
      */
-    private static class VisorCacheMetricsJob extends VisorJob<IgniteBiTuple<Boolean, String>, Collection<VisorCacheMetrics2>> {
+    private static class VisorCacheMetricsJob
+        extends VisorJob<IgniteBiTuple<Boolean, String>, Map<String, VisorCacheMetrics>> {
         /** */
         private static final long serialVersionUID = 0L;
 
@@ -128,35 +80,15 @@ public class VisorCacheMetricsCollectorTask extends VisorMultiNodeTask<IgniteBiT
         }
 
         /** {@inheritDoc} */
-        @Override protected Collection<VisorCacheMetrics2> run(IgniteBiTuple<Boolean, String> arg) throws IgniteCheckedException {
+        @Override protected Map<String, VisorCacheMetrics>
+            run(IgniteBiTuple<Boolean, String> arg) throws IgniteCheckedException {
             Collection<? extends GridCache<?, ?>> caches = arg.get1() ? g.cachesx() : F.asList(g.cachex(arg.get2()));
 
             if (caches != null) {
-                Collection<VisorCacheMetrics2> res = new ArrayList<>(caches.size());
-
-                for (GridCache<?, ?> c : caches) {
-                    ClusterNodeMetrics m = g.localNode().metrics();
-                    GridCacheMetrics cm = c.metrics();
-                    GridCacheQueryMetrics qm = c.queries().metrics();
-
-                    res.add(new VisorCacheMetrics2(
-                        c.name(),
-                        g.localNode().id(),
-                        m.getTotalCpus(),
-                        (double)m.getHeapMemoryUsed() / m.getHeapMemoryMaximum() * 100.0,
-                        m.getCurrentCpuLoad() * 100.0,
-                        m.getUpTime(),
-                        c.size(),
-                        cm.readTime(),
-                        cm.writeTime(),
-                        cm.hits(),
-                        cm.misses(),
-                        cm.reads(),
-                        cm.writes(),
-                        new VisorCacheQueryMetrics(qm.minimumTime(), qm.maximumTime(), qm.averageTime(),
-                            qm.executions(), qm.fails())
-                    ));
-                }
+                Map<String, VisorCacheMetrics> res = U.newHashMap(caches.size());
+
+                for (GridCache<?, ?> c : caches)
+                    res.put(c.name(), VisorCacheMetrics.from(c));
 
                 return res;
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3032bee8/modules/core/src/main/java/org/gridgain/grid/kernal/visor/cache/VisorCacheQueryAggregatedMetrics.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/visor/cache/VisorCacheQueryAggregatedMetrics.java b/modules/core/src/main/java/org/gridgain/grid/kernal/visor/cache/VisorCacheQueryAggregatedMetrics.java
deleted file mode 100644
index 45f504a..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/visor/cache/VisorCacheQueryAggregatedMetrics.java
+++ /dev/null
@@ -1,129 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.kernal.visor.cache;
-
-import org.gridgain.grid.util.typedef.internal.*;
-
-import java.io.*;
-
-/**
- * Data transfer object for aggregated cache query metrics.
- */
-public class VisorCacheQueryAggregatedMetrics implements Serializable {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** Minimum execution time of query. */
-    private long minTime = Long.MAX_VALUE;
-
-    /** Maximum execution time of query. */
-    private long maxTime = Long.MIN_VALUE;
-
-    /** Average execution time of query. */
-    private double avgTime;
-
-    /** Total execution time of query. */
-    private long totalTime;
-
-    /** Number of executions. */
-    private int execs;
-
-    /** Total number of times a query execution failed. */
-    private int fails;
-
-    /**
-     * @return Minimum execution time of query.
-     */
-    public long minTime() {
-        return minTime;
-    }
-
-    /**
-     * @param minTime New min time.
-     */
-    public void minTime(long minTime) {
-        this.minTime = Math.min(this.minTime, minTime);
-    }
-
-    /**
-     * @return Maximum execution time of query.
-     */
-    public long maxTime() {
-        return maxTime;
-    }
-
-    /**
-     * @param maxTime New max time.
-     */
-    public void maxTime(long maxTime) {
-        this.maxTime = Math.max(this.maxTime, maxTime);
-    }
-
-    /**
-     * @return Average execution time of query.
-     */
-    public double avgTime() {
-        return avgTime;
-    }
-
-    /**
-     * @param avgTime New avg time.
-     */
-    public void avgTime(double avgTime) {
-        this.avgTime = avgTime;
-    }
-
-    /**
-     * @return Total execution time of query.
-     */
-    public long totalTime() {
-        return totalTime;
-    }
-
-    /**
-     * @param totalTime New total time.
-     */
-    public void totalTime(long totalTime) {
-        this.totalTime = totalTime;
-    }
-
-    /**
-     * @return Number of executions.
-     */
-    public int execs() {
-        return execs;
-    }
-
-    /**
-     * @param execs New execs.
-     */
-    public void execs(int execs) {
-        this.execs = execs;
-    }
-
-    /**
-     * @return Total number of times a query execution failed.
-     */
-    public int fails() {
-        return fails;
-    }
-
-    /**
-     * @param fails New fails.
-     */
-    public void fails(int fails) {
-        this.fails = fails;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(VisorCacheQueryAggregatedMetrics.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3032bee8/modules/core/src/main/java/org/gridgain/grid/kernal/visor/cache/VisorCacheQueryMetrics.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/visor/cache/VisorCacheQueryMetrics.java b/modules/core/src/main/java/org/gridgain/grid/kernal/visor/cache/VisorCacheQueryMetrics.java
index 048593d..4418b72 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/visor/cache/VisorCacheQueryMetrics.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/visor/cache/VisorCacheQueryMetrics.java
@@ -9,6 +9,7 @@
 
 package org.gridgain.grid.kernal.visor.cache;
 
+import org.gridgain.grid.cache.query.*;
 import org.gridgain.grid.util.typedef.internal.*;
 
 import java.io.*;
@@ -21,62 +22,61 @@ public class VisorCacheQueryMetrics implements Serializable {
     private static final long serialVersionUID = 0L;
 
     /** Minimum execution time of query. */
-    private final long minTime;
+    private long minTime;
 
     /** Maximum execution time of query. */
-    private final long maxTime;
+    private long maxTime;
 
     /** Average execution time of query. */
-    private final double avgTime;
+    private double avgTime;
 
     /** Number of executions. */
-    private final int execs;
+    private int execs;
 
     /** Total number of times a query execution failed. */
-    private final int fails;
+    private int fails;
 
     /**
-     * Create data transfer object with given parameters.
-     *
-     * @param minTime Minimum execution time of query.
-     * @param maxTime Maximum execution time of query.
-     * @param avgTime Average execution time of query.
-     * @param execs Number of executions.
-     * @param fails Total number of times a query execution failed.
+     * @param m Cache query metrics.
+     * @return Data transfer object for given cache metrics.
      */
-    public VisorCacheQueryMetrics(long minTime, long maxTime, double avgTime, int execs, int fails) {
-        this.minTime = minTime;
-        this.maxTime = maxTime;
-        this.avgTime = avgTime;
-        this.execs = execs;
-        this.fails = fails;
+    public static VisorCacheQueryMetrics from(GridCacheQueryMetrics 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;
     }
 
     /**
      * @return Minimum execution time of query.
      */
-    public long minTime() {
+    public long minimumTime() {
         return minTime;
     }
 
     /**
      * @return Maximum execution time of query.
      */
-    public long maxTime() {
+    public long maximumTime() {
         return maxTime;
     }
 
     /**
      * @return Average execution time of query.
      */
-    public double avgTime() {
+    public double averageTime() {
         return avgTime;
     }
 
     /**
      * @return Number of executions.
      */
-    public int execs() {
+    public int executions() {
         return execs;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3032bee8/modules/core/src/main/java/org/gridgain/grid/kernal/visor/node/VisorGridConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/visor/node/VisorGridConfiguration.java b/modules/core/src/main/java/org/gridgain/grid/kernal/visor/node/VisorGridConfiguration.java
index dba181c..cfb986f 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/visor/node/VisorGridConfiguration.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/visor/node/VisorGridConfiguration.java
@@ -82,7 +82,7 @@ public class VisorGridConfiguration implements Serializable {
      * @param g Grid.
      * @return Fill data transfer object with node configuration data.
      */
-    public VisorGridConfiguration fill(GridEx g) {
+    public VisorGridConfiguration from(GridEx g) {
         assert g != null;
 
         IgniteConfiguration c = g.configuration();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3032bee8/modules/core/src/main/java/org/gridgain/grid/kernal/visor/node/VisorNodeConfigurationCollectorJob.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/visor/node/VisorNodeConfigurationCollectorJob.java b/modules/core/src/main/java/org/gridgain/grid/kernal/visor/node/VisorNodeConfigurationCollectorJob.java
index eade4c9..75ee24b 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/visor/node/VisorNodeConfigurationCollectorJob.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/visor/node/VisorNodeConfigurationCollectorJob.java
@@ -20,7 +20,7 @@ public class VisorNodeConfigurationCollectorJob extends VisorJob<Void, VisorGrid
 
     /** {@inheritDoc} */
     @Override protected VisorGridConfiguration run(Void arg) {
-        return new VisorGridConfiguration().fill(g);
+        return new VisorGridConfiguration().from(g);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3032bee8/modules/visor-console/src/main/scala/org/gridgain/visor/commands/cache/VisorCacheCommand.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/gridgain/visor/commands/cache/VisorCacheCommand.scala b/modules/visor-console/src/main/scala/org/gridgain/visor/commands/cache/VisorCacheCommand.scala
index 1125831..cddc075 100644
--- a/modules/visor-console/src/main/scala/org/gridgain/visor/commands/cache/VisorCacheCommand.scala
+++ b/modules/visor-console/src/main/scala/org/gridgain/visor/commands/cache/VisorCacheCommand.scala
@@ -11,7 +11,7 @@
 
 package org.gridgain.visor.commands.cache
 
-import org.gridgain.grid.kernal.visor.cache.{VisorCacheAggregatedMetrics, VisorCacheConfiguration, VisorCacheMetrics2, VisorCacheMetricsCollectorTask}
+import org.gridgain.grid.kernal.visor.cache._
 import org.gridgain.grid.kernal.visor.node.{VisorGridConfiguration, VisorNodeConfigurationCollectorTask}
 import org.gridgain.grid.util.typedef._
 
@@ -285,7 +285,7 @@ class VisorCacheCommand {
 
                 sumT #= (("Name(@),", "Last Read/Write"), "Nodes", "Entries", "Hits", "Misses", "Reads", "Writes")
 
-                sortAggregatedData(aggrData, sortType getOrElse "lr", reversed).foreach(
+                sortAggregatedData(aggrData, sortType.getOrElse("lr"), reversed).foreach(
                     ad => {
                         // Add cache host as visor variable.
                         registerCacheName(ad.cacheName)
@@ -299,29 +299,29 @@ class VisorCacheCommand {
                                 ),
                             ad.nodes,
                             (
-                                "min: " + ad.minSize,
-                                "avg: " + formatDouble(ad.avgSize),
-                                "max: " + ad.maxSize
+                                "min: " + ad.minimumSize,
+                                "avg: " + formatDouble(ad.averageSize),
+                                "max: " + ad.maximumSize
                                 ),
                             (
-                                "min: " + ad.minHits,
-                                "avg: " + formatDouble(ad.avgHits),
-                                "max: " + ad.maxHits
+                                "min: " + ad.minimumHits,
+                                "avg: " + formatDouble(ad.averageHits),
+                                "max: " + ad.maximumHits
                                 ),
                             (
-                                "min: " + ad.minMisses,
-                                "avg: " + formatDouble(ad.avgMisses),
-                                "max: " + ad.maxMisses
+                                "min: " + ad.minimumMisses,
+                                "avg: " + formatDouble(ad.averageMisses),
+                                "max: " + ad.maximumMisses
                                 ),
                             (
-                                "min: " + ad.minReads,
-                                "avg: " + formatDouble(ad.avgReads),
-                                "max: " + ad.maxReads
+                                "min: " + ad.minimumReads,
+                                "avg: " + formatDouble(ad.averageReads),
+                                "max: " + ad.maximumReads
                                 ),
                             (
-                                "min: " + ad.minWrites,
-                                "avg: " + formatDouble(ad.avgWrites),
-                                "max: " + ad.maxWrites
+                                "min: " + ad.minimumWrites,
+                                "avg: " + formatDouble(ad.averageWrites),
+                                "max: " + ad.maximumWrites
                                 )
                             )
                     }
@@ -347,38 +347,42 @@ class VisorCacheCommand {
 
                         println("\nCache '" + cacheNameVar + "':")
 
+                        val m = ad.metrics()
+
                         val csT = VisorTextTable()
 
                         csT += ("Name(@)", cacheNameVar)
-                        csT += ("Nodes", ad.nodes.size)
-                        csT += ("Size Min/Avg/Max", ad.minSize + " / " + formatDouble(ad.avgSize) + " / " + ad.maxSize)
+                        csT += ("Nodes", m.size())
+                        csT += ("Size Min/Avg/Max", ad.minimumSize + " / " + formatDouble(ad.averageSize) + " / " + ad.maximumSize)
 
                         val ciT = VisorTextTable()
 
                         ciT #= ("Node ID8(@), IP", "CPUs", "Heap Used", "CPU Load", "Up Time", "Size",
                             "Last Read/Write", "Hi/Mi/Rd/Wr")
 
-                        sortData(ad.metrics(), sortType getOrElse "lr", reversed).
-                            foreach(cd => {
-                                ciT += (
-                                    nodeId8Addr(cd.nodeId),
-                                    cd.cpus,
-                                    formatDouble(cd.heapUsed) + " %",
-                                    formatDouble(cd.cpuLoad) + " %",
-                                    X.timeSpan2HMSM(cd.upTime),
-                                    cd.size,
-                                    (
-                                        formatDateTime(cd.lastRead),
-                                        formatDateTime(cd.lastWrite)
-                                        ),
-                                    (
-                                        "Hi: " + cd.hits,
-                                        "Mi: " + cd.misses,
-                                        "Rd: " + cd.reads,
-                                        "Wr: " + cd.writes
-                                        )
-                                    )
-                            })
+                        sortData(m.toMap, sortType.getOrElse("lr"), reversed).foreach { case (nid, cm) => {
+                            val nm = grid.node(nid).metrics()
+
+                            ciT += (
+                                nodeId8Addr(nid),
+                                nm.getTotalCpus,
+                                formatDouble(nm.getHeapMemoryUsed() / nm.getHeapMemoryMaximum() * 100.0d) + " %",
+
+                                formatDouble(nm.getCurrentCpuLoad * 100.0) + " %",
+                                X.timeSpan2HMSM(nm.getUpTime),
+                                cm.size(),
+                                (
+                                    formatDateTime(cm.readTime),
+                                    formatDateTime(cm.writeTime)
+                                ),
+                                (
+                                    "Hi: " + cm.hits(),
+                                    "Mi: " + cm.misses(),
+                                    "Rd: " + cm.reads(),
+                                    "Wr: " + cm.writes()
+                                )
+                            )
+                        }}
 
                         csT.render()
 
@@ -394,15 +398,13 @@ class VisorCacheCommand {
                         println("'Wr' - Number of cache writes.")
 
                         // Print metrics.
-                        val qm = ad.queryMetrics()
-
                         nl()
                         println("Aggregated queries metrics:")
-                        println("  Minimum execution time: " + X.timeSpan2HMSM(qm.minTime))
-                        println("  Maximum execution time: " + X.timeSpan2HMSM(qm.maxTime))
-                        println("  Average execution time: " + X.timeSpan2HMSM(qm.avgTime.toLong))
-                        println("  Total number of executions: " + qm.execs)
-                        println("  Total number of failures:   " + qm.fails)
+                        println("  Minimum execution time: " + X.timeSpan2HMSM(ad.minimumQueryTime()))
+                        println("  Maximum execution time: " + X.timeSpan2HMSM(ad.maximumQueryTime))
+                        println("  Average execution time: " + X.timeSpan2HMSM(ad.averageQueryTime.toLong))
+                        println("  Total number of executions: " + ad.execsQuery)
+                        println("  Total number of failures:   " + ad.failsQuery)
 
                         gCfg.foreach(_.caches().find(_.name() == ad.cacheName()).foreach(cfg => {
                             nl()
@@ -514,19 +516,18 @@ class VisorCacheCommand {
      * @param reverse Whether to reverse sorting or not.
      * @return Sorted data.
      */
-    private def sortData(data: Iterable[VisorCacheMetrics2], arg: String, reverse: Boolean): List[VisorCacheMetrics2] = {
+    private def sortData(data: Map[UUID, VisorCacheMetrics], arg: String, reverse: Boolean) = {
         assert(data != null)
         assert(arg != null)
 
         val sorted = arg.trim match {
-            case "lr" => data.toList.sortBy(_.lastRead)
-            case "lw" => data.toList.sortBy(_.lastWrite)
-            case "hi" => data.toList.sortBy(_.hits)
-            case "mi" => data.toList.sortBy(_.misses)
-            case "rd" => data.toList.sortBy(_.reads)
-            case "wr" => data.toList.sortBy(_.writes)
-            case "cn" => data.toList.sortWith((x, y) => x.cacheName == null ||
-                x.cacheName.toLowerCase < y.cacheName.toLowerCase)
+            case "lr" => data.toSeq.sortBy(_._2.readTime)
+            case "lw" => data.toSeq.sortBy(_._2.writeTime)
+            case "hi" => data.toSeq.sortBy(_._2.hits)
+            case "mi" => data.toSeq.sortBy(_._2.misses)
+            case "rd" => data.toSeq.sortBy(_._2.reads)
+            case "wr" => data.toSeq.sortBy(_._2.writes)
+            case "cn" => data.toSeq.sortBy(_._1)
 
             case _ =>
                 assert(false, "Unknown sorting type: " + arg)
@@ -551,10 +552,10 @@ class VisorCacheCommand {
         val sorted = arg.trim match {
             case "lr" => data.toList.sortBy(_.lastRead)
             case "lw" => data.toList.sortBy(_.lastWrite)
-            case "hi" => data.toList.sortBy(_.avgHits)
-            case "mi" => data.toList.sortBy(_.avgMisses)
-            case "rd" => data.toList.sortBy(_.avgReads)
-            case "wr" => data.toList.sortBy(_.avgWrites)
+            case "hi" => data.toList.sortBy(_.averageHits)
+            case "mi" => data.toList.sortBy(_.averageMisses)
+            case "rd" => data.toList.sortBy(_.averageReads)
+            case "wr" => data.toList.sortBy(_.averageWrites)
             case "cn" => data.toList.sortWith((x, y) =>
                 x.cacheName == null || (y.cacheName != null && x.cacheName.toLowerCase < y.cacheName.toLowerCase))
 
@@ -607,9 +608,9 @@ class VisorCacheCommand {
                     ),
                 ad.nodes,
                 (
-                    "min: " + ad.minSize,
-                    "avg: " + formatDouble(ad.avgSize),
-                    "max: " + ad.maxSize
+                    "min: " + ad.minimumSize,
+                    "avg: " + formatDouble(ad.averageSize),
+                    "max: " + ad.maximumSize
                 ))
         })