From commits-return-121685-archive-asf-public=cust-asf.ponee.io@ignite.apache.org Thu Nov 15 08:25:00 2018 Return-Path: X-Original-To: archive-asf-public@cust-asf.ponee.io Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx-eu-01.ponee.io (Postfix) with SMTP id D9C6E180669 for ; Thu, 15 Nov 2018 08:24:58 +0100 (CET) Received: (qmail 82528 invoked by uid 500); 15 Nov 2018 07:24:57 -0000 Mailing-List: contact commits-help@ignite.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@ignite.apache.org Delivered-To: mailing list commits@ignite.apache.org Received: (qmail 82519 invoked by uid 99); 15 Nov 2018 07:24:57 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 15 Nov 2018 07:24:57 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id B90ACE1141; Thu, 15 Nov 2018 07:24:57 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: akuznetsov@apache.org To: commits@ignite.apache.org Message-Id: X-Mailer: ASF-Git Admin Mailer Subject: ignite git commit: IGNITE-336 Visor CMD: Added commands to enable/disable cache statistics in runtime. Date: Thu, 15 Nov 2018 07:24:57 +0000 (UTC) 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 Authored: Thu Nov 15 14:12:36 2018 +0700 Committer: Alexey Kuznetsov 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 { + /** */ + 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 { + /** */ + 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 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 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 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 -slp -c= * cache -rlp -c= + * cache -statistics= -c= * }}} * * ====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= * 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 -rebalance -c=", "cache -slp -c=", - "cache -rlp -c=" + "cache -rlp -c=", + "cache -statistics= -c=" ), args = Seq( "-id8=" -> 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=" -> "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= -c= + * }}} + * + * ====Arguments==== + * {{{ + * + * Statistics collection state to set for cache. + * + * 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=== + * cache -statistics=on -c=cache + * Enable collection of statistics for cache with name 'cache'. + * cache -statistics=off -c=cache + * 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; } }