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 2018/11/15 07:24:57 UTC

ignite git commit: IGNITE-336 Visor CMD: Added commands to enable/disable cache statistics in runtime.

Repository: ignite
Updated Branches:
  refs/heads/master d196ce355 -> bf2f05e32


IGNITE-336 Visor CMD: Added commands to enable/disable cache statistics in runtime.


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

Branch: refs/heads/master
Commit: bf2f05e323d7eb6fbc0a1c5b9111ed8c05be503d
Parents: d196ce3
Author: Vasiliy Sisko <vs...@gridgain.com>
Authored: Thu Nov 15 14:12:36 2018 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Thu Nov 15 14:12:36 2018 +0700

----------------------------------------------------------------------
 .../ignite/internal/visor/cache/VisorCache.java |  20 ++-
 .../cache/VisorCacheToggleStatisticsTask.java   |  72 +++++++++
 .../VisorCacheToggleStatisticsTaskArg.java      |  87 +++++++++++
 .../commands/cache/VisorCacheCommand.scala      |  76 ++++++----
 .../VisorCacheToggleStatisticsCommand.scala     | 152 +++++++++++++++++++
 .../app/modules/cluster/CacheMetrics.js         |   3 +
 6 files changed, 380 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/bf2f05e3/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCache.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCache.java
index 63eb13c..495a9cb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCache.java
@@ -91,6 +91,9 @@ public class VisorCache extends VisorDataTransferObject {
     /** Cache system state. */
     private boolean sys;
 
+    /** Checks whether statistics collection is enabled in this cache. */
+    private boolean statisticsEnabled;
+
     /**
      * Create data transfer object for given cache.
      */
@@ -119,7 +122,7 @@ public class VisorCache extends VisorDataTransferObject {
         backupSize = ca.localSizeLong(PEEK_ONHEAP_BACKUP);
         nearSize = ca.nearSize();
         size = primarySize + backupSize + nearSize;
-        
+
         partitions = ca.affinity().partitions();
         near = cctx.isNear();
 
@@ -127,6 +130,8 @@ public class VisorCache extends VisorDataTransferObject {
             metrics = new VisorCacheMetrics(ignite, name);
 
         sys = ignite.context().cache().systemCache(name);
+
+        statisticsEnabled = ca.clusterMetrics().isStatisticsEnabled();
     }
 
     /**
@@ -278,9 +283,16 @@ public class VisorCache extends VisorDataTransferObject {
         return metrics != null ? metrics.getOffHeapEntriesCount() : 0L;
     }
 
+    /**
+     * @return Checks whether statistics collection is enabled in this cache.
+     */
+    public boolean isStatisticsEnabled() {
+        return statisticsEnabled;
+    }
+
     /** {@inheritDoc} */
     @Override public byte getProtocolVersion() {
-        return V2;
+        return V3;
     }
 
     /** {@inheritDoc} */
@@ -298,6 +310,7 @@ public class VisorCache extends VisorDataTransferObject {
         out.writeBoolean(near);
         out.writeObject(metrics);
         out.writeBoolean(sys);
+        out.writeBoolean(statisticsEnabled);
     }
 
     /** {@inheritDoc} */
@@ -316,6 +329,9 @@ public class VisorCache extends VisorDataTransferObject {
         metrics = (VisorCacheMetrics)in.readObject();
 
         sys = protoVer > V1 ? in.readBoolean() : metrics != null && metrics.isSystem();
+
+        if (protoVer > V2)
+            statisticsEnabled = in.readBoolean();
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf2f05e3/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheToggleStatisticsTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheToggleStatisticsTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheToggleStatisticsTask.java
new file mode 100644
index 0000000..aebed81
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheToggleStatisticsTask.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.visor.cache;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.task.GridInternal;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.visor.VisorJob;
+import org.apache.ignite.internal.visor.VisorOneNodeTask;
+
+/**
+ * Switch statisticsEnabled flag for specified caches to specified state.
+ */
+@GridInternal
+public class VisorCacheToggleStatisticsTask extends VisorOneNodeTask<VisorCacheToggleStatisticsTaskArg, Void> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** {@inheritDoc} */
+    @Override protected VisorCachesToggleStatisticsJob job(VisorCacheToggleStatisticsTaskArg arg) {
+        return new VisorCachesToggleStatisticsJob(arg, debug);
+    }
+
+    /**
+     * Job that switch statisticsEnabled flag for specified caches to specified state.
+     */
+    private static class VisorCachesToggleStatisticsJob extends VisorJob<VisorCacheToggleStatisticsTaskArg, Void> {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /**
+         * @param arg Job argument object.
+         * @param debug Debug flag.
+         */
+        private VisorCachesToggleStatisticsJob(VisorCacheToggleStatisticsTaskArg arg, boolean debug) {
+            super(arg, debug);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected Void run(VisorCacheToggleStatisticsTaskArg arg) {
+            try {
+                ignite.context().cache().enableStatistics(arg.getCacheNames(), arg.getState());
+            }
+            catch (IgniteCheckedException e) {
+                throw U.convertException(e);
+            }
+
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(VisorCachesToggleStatisticsJob.class, this);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf2f05e3/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheToggleStatisticsTaskArg.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheToggleStatisticsTaskArg.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheToggleStatisticsTaskArg.java
new file mode 100644
index 0000000..34359da
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheToggleStatisticsTaskArg.java
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.visor.cache;
+
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.util.Set;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.visor.VisorDataTransferObject;
+
+/**
+ * Argument for {@link VisorCacheToggleStatisticsTask}.
+ */
+public class VisorCacheToggleStatisticsTaskArg extends VisorDataTransferObject {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** State to set to statisticsEnabled flag. */
+    private boolean state;
+
+    /** Cache names to toggle statisticsEnabled flag. */
+    private Set<String> cacheNames;
+
+    /**
+     * Default constructor.
+     */
+    public VisorCacheToggleStatisticsTaskArg() {
+        // No-op.
+    }
+
+    /**
+     * @param state State to set to statisticsEnabled flag.
+     * @param cacheNames Collection of cache names to toggle statisticsEnabled flag.
+     */
+    public VisorCacheToggleStatisticsTaskArg(boolean state, Set<String> cacheNames) {
+        this.state = state;
+        this.cacheNames = cacheNames;
+    }
+
+    /**
+     * @return State to set to statisticsEnabled flag.
+     */
+    public boolean getState() {
+        return state;
+    }
+
+    /**
+     * @return Cache names to toggle statisticsEnabled flag.
+     */
+    public Set<String> getCacheNames() {
+        return cacheNames;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void writeExternalData(ObjectOutput out) throws IOException {
+        out.writeBoolean(state);
+        U.writeCollection(out, cacheNames);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException, ClassNotFoundException {
+        state = in.readBoolean();
+        cacheNames = U.readSet(in);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(VisorCacheToggleStatisticsTaskArg.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf2f05e3/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
index 25f1212..5853610 100755
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
@@ -41,31 +41,33 @@ import scala.language.{implicitConversions, reflectiveCalls}
  *
  * ==Help==
  * {{{
- * +-------------------------------------------------------------------------------------------+
- * | cache            | Prints statistics about caches from specified node on the entire grid. |
- * |                  | Output sorting can be specified in arguments.                          |
- * |                  |                                                                        |
- * |                  | Output abbreviations:                                                  |
- * |                  |     #   Number of nodes.                                               |
- * |                  |     H/h Number of cache hits.                                          |
- * |                  |     M/m Number of cache misses.                                        |
- * |                  |     R/r Number of cache reads.                                         |
- * |                  |     W/w Number of cache writes.                                        |
- * +-------------------------------------------------------------------------------------------+
- * | cache -clear     | Clears all entries from cache on all nodes.                            |
- * +-------------------------------------------------------------------------------------------+
- * | cache -scan      | List all entries in cache with specified name.                         |
- * +-------------------------------------------------------------------------------------------+
- * | cache -stop      | Stop cache with specified name.                                        |
- * +-------------------------------------------------------------------------------------------+
- * | cache -reset     | Reset metrics for cache with specified name.                           |
- * +-------------------------------------------------------------------------------------------+
- * | cache -rebalance | Re-balance partitions for cache with specified name.                   |
- * +-------------------------------------------------------------------------------------------+
- * | cache -slp       | Show list of lost partitions for specified cache.                      |
- * +-------------------------------------------------------------------------------------------+
- * | cache -rlp       | Reset lost partitions for specified cache.                             |
- * +-------------------------------------------------------------------------------------------+
+ * +--------------------------------------------------------------------------------------------+
+ * | cache             | Prints statistics about caches from specified node on the entire grid. |
+ * |                   | Output sorting can be specified in arguments.                          |
+ * |                   |                                                                        |
+ * |                   | Output abbreviations:                                                  |
+ * |                   |     #   Number of nodes.                                               |
+ * |                   |     H/h Number of cache hits.                                          |
+ * |                   |     M/m Number of cache misses.                                        |
+ * |                   |     R/r Number of cache reads.                                         |
+ * |                   |     W/w Number of cache writes.                                        |
+ * +--------------------------------------------------------------------------------------------+
+ * | cache -clear      | Clears all entries from cache on all nodes.                            |
+ * +--------------------------------------------------------------------------------------------+
+ * | cache -scan       | List all entries in cache with specified name.                         |
+ * +--------------------------------------------------------------------------------------------+
+ * | cache -stop       | Stop cache with specified name.                                        |
+ * +--------------------------------------------------------------------------------------------+
+ * | cache -reset      | Reset metrics for cache with specified name.                           |
+ * +--------------------------------------------------------------------------------------------+
+ * | cache -rebalance  | Re-balance partitions for cache with specified name.                   |
+ * +--------------------------------------------------------------------------------------------+
+ * | cache -slp        | Show list of lost partitions for specified cache.                      |
+ * +--------------------------------------------------------------------------------------------+
+ * | cache -rlp        | Reset lost partitions for specified cache.                             |
+ * +--------------------------------------------------------------------------------------------+
+ * | cache -statistics | Switch collection of statistics for specified cache.                   |
+ * +--------------------------------------------------------------------------------------------+
  *
  * }}}
  *
@@ -81,6 +83,7 @@ import scala.language.{implicitConversions, reflectiveCalls}
  *     cache -rebalance -c=<cache-name>
  *     cache -slp -c=<cache-name>
  *     cache -rlp -c=<cache-name>
+ *     cache -statistics=<on|off> -c=<cache-name>
  * }}}
  *
  * ====Arguments====
@@ -133,6 +136,8 @@ import scala.language.{implicitConversions, reflectiveCalls}
  *          Show list of lost partitions for specified cache.
  *     -rlp
  *          Reset lost partitions for specified cache.
+ *     -statistics=<-on|-off>
+ *          Switch collection of statistics for specified cache.
  *     -p=<page size>
  *         Number of object to fetch from cache at once.
  *         Valid range from 1 to 100.
@@ -177,6 +182,10 @@ import scala.language.{implicitConversions, reflectiveCalls}
  *         Show list of lost partitions for cache with name 'cache'.
  *     cache -rlp -c=cache
  *         Reset lost partitions for cache with name 'cache'.
+ *     cache -statistics=on -c=cache
+ *         Enable statistics for cache with name 'cache'.
+ *     cache -statistics=off -c=cache
+ *         Disable statistics for cache with name 'cache'.
  *
  * }}}
  */
@@ -279,7 +288,8 @@ class VisorCacheCommand extends VisorConsoleCommand {
             // Get cache stats data from all nodes.
             val aggrData = cacheData(node, cacheName, showSystem)
 
-            if (hasArgFlagIn("clear", "scan", "stop", "reset", "rebalance", "slp", "rlp")) {
+            if (hasArgFlagIn("clear", "scan", "stop", "reset", "rebalance", "slp", "rlp")
+                || hasArgName("statistics", argLst)) {
                 if (cacheName.isEmpty)
                     askForCache("Select cache from:", node, showSystem
                         && !hasArgFlagIn("clear", "stop", "reset", "rebalance"), aggrData) match {
@@ -309,6 +319,8 @@ class VisorCacheCommand extends VisorConsoleCommand {
                                     VisorCacheLostPartitionsCommand().showLostPartitions(argLst, node)
                                 else if (hasArgFlag("rlp", argLst))
                                     VisorCacheResetLostPartitionsCommand().resetLostPartitions(argLst, node)
+                                else if (hasArgName("statistics", argLst))
+                                    VisorCacheToggleStatisticsCommand().toggle(argLst, node)
                             }
                             else {
                                 if (hasArgFlag("scan", argLst))
@@ -321,6 +333,8 @@ class VisorCacheCommand extends VisorConsoleCommand {
                                     warn("Reset metrics of system cache is not allowed: " + name)
                                 else if (hasArgFlag("rebalance", argLst))
                                     warn("Re-balance partitions of system cache is not allowed: " + name)
+                                else if (hasArgName("statistics", argLst))
+                                    warn("Toggle of statistics collection for system cache is not allowed: " + name)
                             }
                         case None =>
                             warn("Cache with specified name not found: " + name)
@@ -755,7 +769,8 @@ object VisorCacheCommand {
             "cache -reset -c=<cache-name>",
             "cache -rebalance -c=<cache-name>",
             "cache -slp -c=<cache-name>",
-            "cache -rlp -c=<cache-name>"
+            "cache -rlp -c=<cache-name>",
+            "cache -statistics=<on|off> -c=<cache-name>"
   ),
         args = Seq(
             "-id8=<node-id>" -> Seq(
@@ -784,6 +799,7 @@ object VisorCacheCommand {
             "-slp" -> "Show list of lost partitions for specified cache.",
             "-rlp" -> "Reset lost partitions for specified cache.",
             "-rebalance" -> "Re-balance partitions for cache with specified name.",
+            "-statistics=<on|off>" -> "Change statistics collection state for cache with specified name.",
             "-s=hi|mi|rd|wr|cn" -> Seq(
                 "Defines sorting type. Sorted by:",
                 "   hi Hits.",
@@ -845,7 +861,11 @@ object VisorCacheCommand {
             "cache -reset -c=@c0" -> "Reset metrics for cache with name taken from 'c0' memory variable.",
             "cache -rebalance -c=cache" -> "Re-balance partitions for cache with name 'cache'.",
             "cache -slp -c=@c0" -> "Show list of lost partitions for cache with name taken from 'c0' memory variable.",
-            "cache -rlp -c=@c0" -> "Reset lost partitions for cache with name taken from 'c0' memory variable."
+            "cache -rlp -c=@c0" -> "Reset lost partitions for cache with name taken from 'c0' memory variable.",
+            "cache -statistics=on -c=@c0" ->
+                "Enable statistics collection for cache with name taken from 'c0' memory variable.",
+            "cache -statistics=off -c=@c0" ->
+                "Disable statistics collection for cache with name taken from 'c0' memory variable."
         ),
         emptyArgs = cmd.cache,
         withArgs = cmd.cache

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf2f05e3/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheToggleStatisticsCommand.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheToggleStatisticsCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheToggleStatisticsCommand.scala
new file mode 100644
index 0000000..2156de0
--- /dev/null
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheToggleStatisticsCommand.scala
@@ -0,0 +1,152 @@
+/*
+ * 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.visor.commands.cache
+
+import java.util.{HashSet => JavaSet}
+
+import org.apache.ignite.cluster.{ClusterGroupEmptyException, ClusterNode}
+import org.apache.ignite.internal.visor.cache.{VisorCacheToggleStatisticsTask, VisorCacheToggleStatisticsTaskArg}
+import org.apache.ignite.internal.visor.util.VisorTaskUtils._
+import org.apache.ignite.visor.visor._
+
+import scala.language.reflectiveCalls
+
+/**
+ * ==Overview==
+ * Visor 'statistics' command implementation.
+ *
+ * ====Specification====
+ * {{{
+ *     cache -statistics=<on|off> -c=<cache name>
+ * }}}
+ *
+ * ====Arguments====
+ * {{{
+ *     <state>
+  *        Statistics collection state to set for cache.
+ *     <cache-name>
+ *         Name of the cache.
+ * }}}
+ *
+ * ====Examples====
+ * {{{
+ *    cache -statistics=on -c=@c0
+ *        Enable collection of statistics for cache with name taken from 'c0' memory variable.
+ *    cache -statistics=off -c=@c0
+ *        Disable collection of statistics for cache with name taken from 'c0' memory variable.
+ * }}}
+ */
+class VisorCacheToggleStatisticsCommand {
+    /**
+     * Prints error message and advise.
+     *
+     * @param errMsgs Error messages.
+     */
+    private def scold(errMsgs: Any*) {
+        assert(errMsgs != null)
+
+        warn(errMsgs: _*)
+        warn("Type 'help cache' to see how to use this command.")
+    }
+
+    private def error(e: Exception) {
+        var cause: Throwable = e
+
+        while (cause.getCause != null)
+            cause = cause.getCause
+
+        scold(cause.getMessage)
+    }
+
+    /**
+     * ===Command===
+     * Toggle statistics collection for cache with specified name.
+     *
+     * ===Examples===
+     * <ex>cache -statistics=on -c=cache</ex>
+     * Enable collection of statistics for cache with name 'cache'.
+     * <ex>cache -statistics=off -c=cache</ex>
+     * Disable collection of statistics for cache with name 'cache'.
+     *
+     * @param argLst Command arguments.
+     */
+    def toggle(argLst: ArgList, node: Option[ClusterNode]) {
+        val cacheArg = argValue("c", argLst)
+
+        val cacheName = cacheArg match {
+            case None => null // default cache.
+
+            case Some(s) if s.startsWith("@") =>
+                warn("Can't find cache variable with specified name: " + s,
+                    "Type 'cache' to see available cache variables."
+                )
+
+                return
+
+            case Some(name) => name
+        }
+
+        val grp = try {
+            groupForDataNode(node, cacheName)
+        }
+        catch {
+            case _: ClusterGroupEmptyException =>
+                scold(messageNodeNotFound(node, cacheName))
+
+                return
+        }
+
+        try {
+            val cacheNames = new JavaSet[String]()
+            cacheNames.add(cacheName)
+
+            val enable = argValue("statistics", argLst) match {
+                case Some(state) if "on".equalsIgnoreCase(state) => true
+                case Some(state) if "off".equalsIgnoreCase(state) => false
+                case _ =>
+                    warn("Goal state for collection of cache statistics is not specified.",
+                        "Use \"on\" and \"off\" value of \"statistics\" argument to toggle collection of cache statistics.")
+
+                    return
+            }
+
+            executeRandom(grp, classOf[VisorCacheToggleStatisticsTask],
+                new VisorCacheToggleStatisticsTaskArg(enable, cacheNames))
+
+            println("Visor successfully " + (if (enable) "enable" else "disable") +
+                " collection of statistics for cache: " + escapeName(cacheName))
+        }
+        catch {
+            case _: ClusterGroupEmptyException => scold(messageNodeNotFound(node, cacheName))
+            case e: Exception => error(e)
+        }
+    }
+}
+
+/**
+ * Companion object that does initialization of the command.
+ */
+object VisorCacheToggleStatisticsCommand {
+    /** Singleton command. */
+    private val cmd = new VisorCacheToggleStatisticsCommand
+
+    /**
+      * Singleton.
+      */
+    def apply() = cmd
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf2f05e3/modules/web-console/frontend/app/modules/cluster/CacheMetrics.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/cluster/CacheMetrics.js b/modules/web-console/frontend/app/modules/cluster/CacheMetrics.js
index 90d3a9b..609b181 100644
--- a/modules/web-console/frontend/app/modules/cluster/CacheMetrics.js
+++ b/modules/web-console/frontend/app/modules/cluster/CacheMetrics.js
@@ -55,5 +55,8 @@ export default class CacheMetrics {
         // Transaction metrics.
         this.commits = m.txCommits;
         this.rollbacks = m.txRollbacks;
+
+        // Admin metrics.
+        this.statisticsEnabled = m.statisticsEnabled;
     }
 }